You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by vi...@apache.org on 2014/01/08 20:45:35 UTC

git commit: ACCUMULO-2032: make GC serve SSL when SSL is turned on. also remove the entrypoint that allowed you to create a thrift server without thinking about SSL.

Updated Branches:
  refs/heads/1.6.0-SNAPSHOT 7cb5db454 -> 133e90a3c


ACCUMULO-2032: make GC serve SSL when SSL is turned on. also remove the entrypoint that allowed you to create a thrift server without thinking about SSL.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/133e90a3
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/133e90a3
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/133e90a3

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 133e90a3c61aace351be7ab447446c9c6ea6b4f0
Parents: 7cb5db4
Author: Michael Berman <mb...@sqrrl.com>
Authored: Mon Dec 16 17:45:50 2013 -0500
Committer: John Vines <vi...@apache.org>
Committed: Wed Jan 8 14:45:21 2014 -0500

----------------------------------------------------------------------
 .../java/org/apache/accumulo/server/util/TServerUtils.java  | 5 -----
 .../java/org/apache/accumulo/gc/SimpleGarbageCollector.java | 9 ++++++---
 .../org/apache/accumulo/test/functional/ZombieTServer.java  | 2 +-
 .../accumulo/test/performance/thrift/NullTserver.java       | 2 +-
 4 files changed, 8 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/133e90a3/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java b/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
index 502cbe3..64bfd42 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TServerUtils.java
@@ -316,11 +316,6 @@ public class TServerUtils {
   }
 
   public static ServerAddress startTServer(HostAndPort address, TProcessor processor, String serverName, String threadName, int numThreads,
-      long timeBetweenThreadChecks, long maxMessageSize) throws TTransportException {
-    return startTServer(address, processor, serverName, threadName, numThreads, timeBetweenThreadChecks, maxMessageSize, null, -1);
-  }
-
-  public static ServerAddress startTServer(HostAndPort address, TProcessor processor, String serverName, String threadName, int numThreads,
       long timeBetweenThreadChecks, long maxMessageSize, SslConnectionParams sslParams, long sslSocketTimeout) throws TTransportException {
     return startTServer(address, new TimedProcessor(processor, serverName, threadName), serverName, threadName, numThreads, timeBetweenThreadChecks,
         maxMessageSize, sslParams, sslSocketTimeout);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/133e90a3/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index fc24010..03519ba 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -42,6 +42,7 @@ import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -65,6 +66,7 @@ import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.util.ServerServices;
+import org.apache.accumulo.core.util.SslConnectionParams;
 import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
@@ -532,12 +534,13 @@ public class SimpleGarbageCollector implements Iface {
   
   private HostAndPort startStatsService() throws UnknownHostException {
     Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(this));
-    int port = ServerConfiguration.getSystemConfiguration(instance).getPort(Property.GC_PORT);
-    long maxMessageSize = ServerConfiguration.getSystemConfiguration(instance).getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
+    AccumuloConfiguration conf = ServerConfiguration.getSystemConfiguration(instance);
+    int port = conf.getPort(Property.GC_PORT);
+    long maxMessageSize = conf.getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
     HostAndPort result = HostAndPort.fromParts(opts.getAddress(), port);
     log.debug("Starting garbage collector listening on " + result);
     try {
-      return TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2, 1000, maxMessageSize).address;
+      return TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2, 1000, maxMessageSize, SslConnectionParams.forServer(conf), 0).address;
     } catch (Exception ex) {
       log.fatal(ex, ex);
       throw new RuntimeException(ex);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/133e90a3/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
index a52651e..e3e4de3 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZombieTServer.java
@@ -99,7 +99,7 @@ public class ZombieTServer {
     final ThriftClientHandler tch = new ThriftClientHandler(instance, watcher);
     Processor<Iface> processor = new Processor<Iface>(tch);
     ServerAddress serverPort = TServerUtils.startTServer(HostAndPort.fromParts("0.0.0.0", port), processor, "ZombieTServer", "walking dead", 2, 1000,
-        10 * 1024 * 1024);
+        10 * 1024 * 1024, null, -1);
     
     String addressString = serverPort.address.toString();
     String zPath = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + addressString;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/133e90a3/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index ce0370b..0591b19 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -243,7 +243,7 @@ public class NullTserver {
     TransactionWatcher watcher = new TransactionWatcher();
     ThriftClientHandler tch = new ThriftClientHandler(HdfsZooInstance.getInstance(), watcher);
     Processor<Iface> processor = new Processor<Iface>(tch);
-    TServerUtils.startTServer(HostAndPort.fromParts("0.0.0.0", opts.port), processor, "NullTServer", "null tserver", 2, 1000, 10 * 1024 * 1024);
+    TServerUtils.startTServer(HostAndPort.fromParts("0.0.0.0", opts.port), processor, "NullTServer", "null tserver", 2, 1000, 10 * 1024 * 1024, null, -1);
     
     HostAndPort addr = HostAndPort.fromParts(InetAddress.getLocalHost().getHostName(), opts.port);