You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/05/09 17:29:10 UTC

[32/50] [abbrv] git commit: ACCUMULO-2583 First stab at setting up the actual wire transfer to the replication peer

ACCUMULO-2583 First stab at setting up the actual wire transfer to the replication peer

Changed a few things in the thrift idl to make a little more sense. Use the protobuf
Status message that we already have as the response object to the replication. This
allows the peer to not have to process the entire message at one time as an optimization (later).

Made a ReplicationClient in the same spirit as the MasterClient, but with a few notable
changes. Since we're operating outside of the local instance, we don't want to make
the same assertions about availability and blocking until we get a connection as is
the case in talking to the master. This pushes a bit of the logic into the replication
setup code, but this is desirable as we can better handle failure/retry logic as we intend.


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

Branch: refs/heads/ACCUMULO-378
Commit: baa7a4f73739e652d72a47873da967eb2b7dd199
Parents: 59ecd21
Author: Josh Elser <el...@apache.org>
Authored: Wed May 7 00:16:42 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed May 7 00:16:42 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/core/client/Connector.java  |    9 +
 .../client/admin/ReplicationOperations.java     |   52 +
 .../core/client/impl/ConnectorImpl.java         |   12 +
 .../accumulo/core/client/impl/MasterClient.java |    1 +
 .../core/client/impl/ReplicationClient.java     |  208 ++
 .../client/impl/ReplicationOperationsImpl.java  |   82 +
 .../core/client/mock/MockConnector.java         |    7 +
 .../client/replication/PeerExistsException.java |   36 +
 .../replication/PeerNotFoundException.java      |   36 +
 .../core/client/replication/ReplicaSystem.java  |   45 +
 .../replication/ReplicaSystemFactory.java       |   61 +
 .../org/apache/accumulo/core/conf/Property.java |    2 +-
 .../core/replication/ReplicationTarget.java     |   50 +-
 .../thrift/RemoteReplicationCoordinator.java    | 1010 ---------
 .../thrift/RemoteReplicationServicer.java       | 1917 ----------------
 .../thrift/ReplicationCoordinator.java          | 1010 +++++++++
 .../replication/thrift/ReplicationServicer.java | 2145 ++++++++++++++++++
 core/src/main/thrift/replication.thrift         |    8 +-
 .../core/replication/AccumuloReplicaSystem.java |  124 +
 .../core/replication/ReplicationTargetTest.java |   13 +-
 .../ReplicationWorkAssignerHelper.java          |   69 +
 .../MasterReplicationCoordinator.java           |    4 +-
 .../replication/ReplicationWorkAssigner.java    |   10 +-
 .../accumulo/master/replication/WorkMaker.java  |    2 +-
 .../ReplicationWorkAssignerTest.java            |    9 +-
 .../replication/ReplicationProcessor.java       |  130 +-
 .../tserver/replication/ReplicationWorker.java  |    2 +-
 .../test/replication/ReplicationWithGCIT.java   |    2 +-
 .../replication/ReplicationWithMakerTest.java   |    2 +-
 29 files changed, 4090 insertions(+), 2968 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/Connector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
index 4a2acff..301577c 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.client;
 
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.NamespaceOperations;
+import org.apache.accumulo.core.client.admin.ReplicationOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.security.Authorizations;
@@ -228,4 +229,12 @@ public abstract class Connector {
    * @return an object to modify instance configuration
    */
   public abstract InstanceOperations instanceOperations();
+
+  /**
+   * Retrieves a ReplicationOperations object to manage replication configuration.
+   * 
+   * @return an object to modify replication configuration
+   * @since 1.7.0
+   */
+  public abstract ReplicationOperations replicationOperations();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
new file mode 100644
index 0000000..22cfcaf
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
@@ -0,0 +1,52 @@
+/*
+ * 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.accumulo.core.client.admin;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.replication.PeerExistsException;
+import org.apache.accumulo.core.client.replication.PeerNotFoundException;
+import org.apache.accumulo.core.client.replication.ReplicaSystem;
+
+/**
+ * Supports replication configuration
+ * @since 1.7.0
+ */
+public interface ReplicationOperations {
+
+  /**
+   * Define a cluster with the given name using the given {@link ReplicaSystem}
+   * @param name Name of the cluster, used for configuring replication on tables
+   * @param system Type of system to be replicated to
+   */
+  public void addPeer(String name, ReplicaSystem system) throws AccumuloException, AccumuloSecurityException, PeerExistsException;
+
+  /**
+   * Define a cluster with the given name and the given name system
+   * @param name Unique name for the cluster
+   * @param replicaType {@link ReplicaSystem} class name to use to replicate the data 
+   * @throws PeerExistsException
+   */
+  public void addPeer(String name, String replicaType) throws AccumuloException, AccumuloSecurityException, PeerExistsException;
+
+  /**
+   * Remove a cluster with the given name
+   * @param name Name of the cluster to remove
+   * @throws PeerNotFoundException
+   */
+  public void removePeer(String name) throws AccumuloException, AccumuloSecurityException, PeerNotFoundException;
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
index 0df35f6..e22be52 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.core.client.impl;
 import java.util.concurrent.TimeUnit;
 
 import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchDeleter;
@@ -35,6 +36,7 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.NamespaceOperations;
+import org.apache.accumulo.core.client.admin.ReplicationOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
@@ -51,6 +53,7 @@ public class ConnectorImpl extends Connector {
   private TableOperations tableops = null;
   private NamespaceOperations namespaceops = null;
   private InstanceOperations instanceops = null;
+  private ReplicationOperations replicationops = null;
 
   public ConnectorImpl(final Instance instance, Credentials cred) throws AccumuloException, AccumuloSecurityException {
     checkArgument(instance != null, "instance is null");
@@ -182,4 +185,13 @@ public class ConnectorImpl extends Connector {
 
     return instanceops;
   }
+
+  @Override
+  public synchronized ReplicationOperations replicationOperations() {
+    if (null == replicationops) {
+      replicationops = new ReplicationOperationsImpl(instance, credentials); 
+    }
+
+    return replicationops;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
index 1fa8f12..9807a82 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.core.client.impl;
 
 import static com.google.common.base.Preconditions.checkArgument;
+
 import java.net.UnknownHostException;
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
new file mode 100644
index 0000000..df12ae8
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
@@ -0,0 +1,208 @@
+/*
+ * 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.accumulo.core.client.impl;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.net.UnknownHostException;
+import java.util.List;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.replication.thrift.ReplicationCoordinator;
+import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
+import org.apache.accumulo.core.util.ThriftUtil;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.thrift.TServiceClient;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReplicationClient {
+  private static final Logger log = LoggerFactory.getLogger(ReplicationClient.class);
+
+  /**
+   * @param instance
+   *          Instance for the peer replicant
+   * @return Client to the ReplicationCoordinator service
+   */
+  public static ReplicationCoordinator.Client getCoordinatorConnectionWithRetry(Instance instance) {
+    checkArgument(instance != null, "instance is null");
+
+    while (true) {
+
+      ReplicationCoordinator.Client result = getCoordinatorConnection(instance);
+      if (result != null)
+        return result;
+      UtilWaitThread.sleep(250);
+    }
+
+  }
+
+  public static ReplicationCoordinator.Client getCoordinatorConnection(Instance instance) {
+    List<String> locations = instance.getMasterLocations();
+
+    if (locations.size() == 0) {
+      log.debug("No masters...");
+      return null;
+    }
+
+    String master = locations.get(0);
+    if (master.endsWith(":0"))
+      return null;
+
+    try {
+      // Master requests can take a long time: don't ever time out
+      ReplicationCoordinator.Client client = ThriftUtil.getClientNoTimeout(new ReplicationCoordinator.Client.Factory(), master,
+          ServerConfigurationUtil.getConfiguration(instance));
+      return client;
+    } catch (TTransportException tte) {
+      if (tte.getCause().getClass().equals(UnknownHostException.class)) {
+        // do not expect to recover from this
+        throw new RuntimeException(tte);
+      }
+      log.debug("Failed to connect to master={}, will retry... ", master, tte);
+      return null;
+    }
+  }
+
+  /**
+   * Attempt a single time to create a ReplicationServicer client to the given host
+   * 
+   * @param inst
+   *          Instance to the peer replicant
+   * @param server
+   *          Server to connect to
+   * @return A ReplicationServicer client to the given host in the given instance
+   */
+  public static ReplicationServicer.Client getServicerConnection(Instance inst, String server) throws TTransportException {
+    checkNotNull(inst);
+    checkNotNull(server);
+
+    try {
+      return ThriftUtil.getClientNoTimeout(new ReplicationServicer.Client.Factory(), server, ServerConfigurationUtil.getConfiguration(inst));
+    } catch (TTransportException tte) {
+      log.debug("Failed to connect to servicer ({}), will retry...", tte);
+      throw tte;
+    }
+  }
+
+  public static void close(ReplicationCoordinator.Iface iface) {
+    TServiceClient client = (TServiceClient) iface;
+    if (client != null && client.getInputProtocol() != null && client.getInputProtocol().getTransport() != null) {
+      ThriftTransportPool.getInstance().returnTransport(client.getInputProtocol().getTransport());
+    } else {
+      log.debug("Attempt to close null connection to the remote system", new Exception());
+    }
+  }
+
+  public static void close(ReplicationServicer.Iface iface) {
+    TServiceClient client = (TServiceClient) iface;
+    if (client != null && client.getInputProtocol() != null && client.getInputProtocol().getTransport() != null) {
+      ThriftTransportPool.getInstance().returnTransport(client.getInputProtocol().getTransport());
+    } else {
+      log.debug("Attempt to close null connection to the remote system", new Exception());
+    }
+  }
+
+  public static <T> T executeCoordinatorWithReturn(Instance instance, ClientExecReturn<T,ReplicationCoordinator.Client> exec) throws AccumuloException,
+      AccumuloSecurityException {
+    ReplicationCoordinator.Client client = null;
+    while (true) {
+      try {
+        client = getCoordinatorConnectionWithRetry(instance);
+        return exec.execute(client);
+      } catch (TTransportException tte) {
+        log.debug("ReplicationClient coordinator request failed, retrying ... ", tte);
+        UtilWaitThread.sleep(100);
+      } catch (ThriftSecurityException e) {
+        throw new AccumuloSecurityException(e.user, e.code, e);
+      } catch (AccumuloException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new AccumuloException(e);
+      } finally {
+        if (client != null)
+          close(client);
+      }
+    }
+  }
+
+  public static void executeCoordinator(Instance instance, ClientExec<ReplicationCoordinator.Client> exec) throws AccumuloException, AccumuloSecurityException {
+    ReplicationCoordinator.Client client = null;
+    try {
+      client = getCoordinatorConnectionWithRetry(instance);
+      exec.execute(client);
+    } catch (TTransportException tte) {
+      log.debug("ReplicationClient coordinator request failed, retrying ... ", tte);
+      UtilWaitThread.sleep(100);
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    } finally {
+      if (client != null)
+        close(client);
+    }
+  }
+
+  public static <T> T executeServicerWithReturn(Instance instance, String tserver, ClientExecReturn<T,ReplicationServicer.Client> exec)
+      throws AccumuloException, AccumuloSecurityException, TTransportException {
+    ReplicationServicer.Client client = null;
+    while (true) {
+      try {
+        client = getServicerConnection(instance, tserver);
+        return exec.execute(client);
+      } catch (ThriftSecurityException e) {
+        throw new AccumuloSecurityException(e.user, e.code, e);
+      } catch (AccumuloException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new AccumuloException(e);
+      } finally {
+        if (client != null)
+          close(client);
+      }
+    }
+  }
+
+  public static void executeServicer(Instance instance, String tserver, ClientExec<ReplicationServicer.Client> exec) throws AccumuloException,
+      AccumuloSecurityException, TTransportException {
+    ReplicationServicer.Client client = null;
+    try {
+      client = getServicerConnection(instance, tserver);
+      exec.execute(client);
+      return;
+    } catch (ThriftSecurityException e) {
+      throw new AccumuloSecurityException(e.user, e.code, e);
+    } catch (AccumuloException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new AccumuloException(e);
+    } finally {
+      if (client != null)
+        close(client);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
new file mode 100644
index 0000000..5bbb1c1
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
@@ -0,0 +1,82 @@
+/*
+ * 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.accumulo.core.client.impl;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.admin.ReplicationOperations;
+import org.apache.accumulo.core.client.replication.PeerExistsException;
+import org.apache.accumulo.core.client.replication.PeerNotFoundException;
+import org.apache.accumulo.core.client.replication.ReplicaSystem;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.master.thrift.MasterClientService.Client;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.trace.instrument.Tracer;
+
+/**
+ * 
+ */
+public class ReplicationOperationsImpl implements ReplicationOperations {
+
+  private Instance inst;
+  private Credentials creds;
+
+  public ReplicationOperationsImpl(Instance inst, Credentials creds) {
+    this.inst = inst;
+    this.creds = creds;
+  }
+
+  @Override
+  public void addPeer(String name, ReplicaSystem system) throws AccumuloException, AccumuloSecurityException, PeerExistsException {
+    checkNotNull(name);
+    checkNotNull(system);
+
+    addPeer(name, system.getClass().getName());
+  }
+
+  @Override
+  public void addPeer(final String name, final String replicaType) throws AccumuloException, AccumuloSecurityException, PeerExistsException {
+    checkNotNull(name);
+    checkNotNull(replicaType);
+
+    MasterClient.execute(inst, new ClientExec<Client>() {
+
+      @Override
+      public void execute(Client client) throws Exception {
+        client.setSystemProperty(Tracer.traceInfo(), creds.toThrift(inst), Property.REPLICATION_PEERS.getKey() + name, replicaType);
+      }
+
+    });
+  }
+
+  @Override
+  public void removePeer(final String name) throws AccumuloException, AccumuloSecurityException, PeerNotFoundException {
+    checkNotNull(name);
+
+    MasterClient.execute(inst, new ClientExec<Client>() {
+
+      @Override
+      public void execute(Client client) throws Exception {
+        client.removeSystemProperty(Tracer.traceInfo(), creds.toThrift(inst), Property.REPLICATION_PEERS.getKey() + name);
+      }
+
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index 996198c..8613602 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
+import org.apache.accumulo.core.client.admin.ReplicationOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
@@ -147,5 +148,11 @@ public class MockConnector extends Connector {
     // TODO add implementation
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public ReplicationOperations replicationOperations() {
+    // TODO add implementation
+    throw new UnsupportedOperationException();
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java
new file mode 100644
index 0000000..d205238
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerExistsException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.accumulo.core.client.replication;
+
+/**
+ * The peer already exists
+ */
+public class PeerExistsException extends Exception {
+  private static final long serialVersionUID = 1L;
+
+  public PeerExistsException(String peer) {
+    this(peer, (String) null);
+  }
+
+  public PeerExistsException(String peer, String message) {
+    super("Peer '" + peer + "' already exists" + (null == message || message.isEmpty() ? "" : message));
+  }
+
+  public PeerExistsException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
new file mode 100644
index 0000000..1859c62
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.accumulo.core.client.replication;
+
+/**
+ * The peer already exists
+ */
+public class PeerNotFoundException extends Exception {
+  private static final long serialVersionUID = 1L;
+
+  public PeerNotFoundException(String peer) {
+    this(peer, (String) null);
+  }
+
+  public PeerNotFoundException(String peer, String message) {
+    super("Peer '" + peer + "' not found " + (null == message || message.isEmpty() ? "" : message));
+  }
+
+  public PeerNotFoundException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystem.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystem.java b/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystem.java
new file mode 100644
index 0000000..220d7bb
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystem.java
@@ -0,0 +1,45 @@
+/*
+ * 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.accumulo.core.client.replication;
+
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.replication.proto.Replication.Status;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Encapsulation of a remote system which Accumulo can replicate data to
+ */
+public interface ReplicaSystem {
+
+  /**
+   * Replicate the given status to the target peer
+   * @param p Path to the resource we're reading from
+   * @param status Information to replicate
+   * @param target The peer
+   * @return A new Status for the progress that was made
+   */
+  public Status replicate(Path p, Status status, ReplicationTarget target);
+
+  /**
+   * Configure the implementation with necessary information from the system configuration
+   * <p>
+   * For example, we only need one implementation for Accumulo, but, for each peer,
+   * we have a ZK quorom and instance name
+   * @param configuration
+   */
+  public void configure(String configuration);
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystemFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystemFactory.java b/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystemFactory.java
new file mode 100644
index 0000000..3aa8ea4
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystemFactory.java
@@ -0,0 +1,61 @@
+/*
+ * 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.accumulo.core.client.replication;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * 
+ */
+public class ReplicaSystemFactory {
+  private static final Logger log = LoggerFactory.getLogger(ReplicaSystemFactory.class);
+
+  /**
+   * @param value {@link ReplicaSystem} implementation class name
+   * @return A {@link ReplicaSystem} object from the given name
+   */
+  public static ReplicaSystem get(String value) {
+    Preconditions.checkNotNull(value);
+
+    int index = value.indexOf(',');
+    if (-1 == index) {
+      throw new IllegalArgumentException("Expected comma separator between replication system name and configuration");
+    }
+
+    String name = value.substring(0, index);
+    String configuration = value.substring(index + 1);
+    
+    try {
+      Class<?> clz = Class.forName(name);
+      Object o = clz.newInstance();
+
+      if (ReplicaSystem.class.isAssignableFrom(o.getClass())) {
+        ReplicaSystem rs = (ReplicaSystem) o;
+        rs.configure(configuration);
+        return rs;
+      }
+
+      throw new RuntimeException("Class is not assignable to ReplicaSystem: " + name);
+    } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) {
+      log.error("Error creating ReplicaSystem object", e);
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 6700736..f31cd46 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -449,7 +449,7 @@ public enum Property {
   @Experimental
   REPLICATION_PREFIX("replication.", null, PropertyType.PREFIX, "Properties in this category affect the replication of data to other Accumulo instances."),
   @Experimental
-  REPLICATION_SLAVES("replication.slave.", null, PropertyType.PREFIX, "Properties in this category control what systems data can be replicated to"),
+  REPLICATION_PEERS("replication.peer.", null, PropertyType.PREFIX, "Properties in this category control what systems data can be replicated to"),
   @Experimental
   REPLICATION_BATCH_SIZE("replication.batch.size", "1000", PropertyType.COUNT, "Maximum number of updates (WAL) or key-value pairs (RFile) to send in one replication task"),
   @Experimental

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
index a75d52a..3a60632 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTarget.java
@@ -31,22 +31,22 @@ import org.apache.hadoop.io.WritableUtils;
  */
 public class ReplicationTarget implements Writable {
 
-  private String remoteName;
+  private String peerName;
   private String remoteIdentifier;
 
   public ReplicationTarget() { }
 
-  public ReplicationTarget(String remoteName, String remoteIdentifier) {
-    this.remoteName = remoteName;
+  public ReplicationTarget(String peerName, String remoteIdentifier) {
+    this.peerName = peerName;
     this.remoteIdentifier = remoteIdentifier;
   }
 
-  public String getRemoteName() {
-    return remoteName;
+  public String getPeerName() {
+    return peerName;
   }
 
-  public void setRemoteName(String remoteName) {
-    this.remoteName = remoteName;
+  public void setPeerName(String peerName) {
+    this.peerName = peerName;
   }
 
   public String getRemoteIdentifier() {
@@ -59,11 +59,11 @@ public class ReplicationTarget implements Writable {
 
   @Override
   public void write(DataOutput out) throws IOException {
-    if (null == remoteName) {
+    if (null == peerName) {
       out.writeBoolean(false);
     } else {
       out.writeBoolean(true);
-      WritableUtils.writeString(out, remoteName);
+      WritableUtils.writeString(out, peerName);
     }
 
     if (null == remoteIdentifier) {
@@ -77,7 +77,7 @@ public class ReplicationTarget implements Writable {
   @Override
   public void readFields(DataInput in) throws IOException {
     if (in.readBoolean()) {
-      this.remoteName = WritableUtils.readString(in);
+      this.peerName = WritableUtils.readString(in);
     }
     if (in.readBoolean()) {
       this.remoteIdentifier = WritableUtils.readString(in);
@@ -87,13 +87,13 @@ public class ReplicationTarget implements Writable {
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder(64);
-    sb.append("Remote Name: ").append(remoteName).append(" Remote identifier: ").append(remoteIdentifier);
+    sb.append("Remote Name: ").append(peerName).append(" Remote identifier: ").append(remoteIdentifier);
     return sb.toString();
   }
 
   @Override
   public int hashCode() {
-    return remoteName.hashCode() ^ remoteIdentifier.hashCode();
+    return peerName.hashCode() ^ remoteIdentifier.hashCode();
   }
 
   @Override
@@ -101,7 +101,7 @@ public class ReplicationTarget implements Writable {
     if (o instanceof ReplicationTarget) {
       ReplicationTarget other = (ReplicationTarget) o;
 
-      return remoteName.equals(other.remoteName) && remoteIdentifier.equals(other.remoteIdentifier);
+      return peerName.equals(other.peerName) && remoteIdentifier.equals(other.remoteIdentifier);
     }
 
     return false;
@@ -127,16 +127,34 @@ public class ReplicationTarget implements Writable {
   }
 
   /**
+   * Deserialize a ReplicationTarget
+   * @param s Serialized copy
+   * @return the deserialized version
+   */
+  public static ReplicationTarget from(String s) {
+    ReplicationTarget target = new ReplicationTarget();
+    DataInputBuffer buffer = new DataInputBuffer();
+    buffer.reset(s.getBytes(), s.length());
+
+    try {
+      target.readFields(buffer);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    return target;
+  }
+
+  /**
    * Convenience method to serialize a ReplicationTarget to {@link Text} using the {@link Writable} methods without caring about
    * performance penalties due to excessive object creation
-   * @param target The object to serialize
    * @return The serialized representation of the object
    */
-  public static Text toText(ReplicationTarget target) {
+  public Text toText() {
     DataOutputBuffer buffer = new DataOutputBuffer();
 
     try {
-      target.write(buffer);
+      this.write(buffer);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/baa7a4f7/core/src/main/java/org/apache/accumulo/core/replication/thrift/RemoteReplicationCoordinator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/thrift/RemoteReplicationCoordinator.java b/core/src/main/java/org/apache/accumulo/core/replication/thrift/RemoteReplicationCoordinator.java
deleted file mode 100644
index 3d07b34..0000000
--- a/core/src/main/java/org/apache/accumulo/core/replication/thrift/RemoteReplicationCoordinator.java
+++ /dev/null
@@ -1,1010 +0,0 @@
-/*
- * 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.
- */
-/**
- * Autogenerated by Thrift Compiler (0.9.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.accumulo.core.replication.thrift;
-
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@SuppressWarnings("all") public class RemoteReplicationCoordinator {
-
-  public interface Iface {
-
-    public String getServicerAddress(int remoteTableId) throws NoServersAvailableException, org.apache.thrift.TException;
-
-  }
-
-  public interface AsyncIface {
-
-    public void getServicerAddress(int remoteTableId, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.getServicerAddress_call> resultHandler) throws org.apache.thrift.TException;
-
-  }
-
-  public static class Client extends org.apache.thrift.TServiceClient implements Iface {
-    public static class Factory implements org.apache.thrift.TServiceClientFactory<Client> {
-      public Factory() {}
-      public Client getClient(org.apache.thrift.protocol.TProtocol prot) {
-        return new Client(prot);
-      }
-      public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
-        return new Client(iprot, oprot);
-      }
-    }
-
-    public Client(org.apache.thrift.protocol.TProtocol prot)
-    {
-      super(prot, prot);
-    }
-
-    public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) {
-      super(iprot, oprot);
-    }
-
-    public String getServicerAddress(int remoteTableId) throws NoServersAvailableException, org.apache.thrift.TException
-    {
-      send_getServicerAddress(remoteTableId);
-      return recv_getServicerAddress();
-    }
-
-    public void send_getServicerAddress(int remoteTableId) throws org.apache.thrift.TException
-    {
-      getServicerAddress_args args = new getServicerAddress_args();
-      args.setRemoteTableId(remoteTableId);
-      sendBase("getServicerAddress", args);
-    }
-
-    public String recv_getServicerAddress() throws NoServersAvailableException, org.apache.thrift.TException
-    {
-      getServicerAddress_result result = new getServicerAddress_result();
-      receiveBase(result, "getServicerAddress");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.e != null) {
-        throw result.e;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getServicerAddress failed: unknown result");
-    }
-
-  }
-  public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
-    public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
-      private org.apache.thrift.async.TAsyncClientManager clientManager;
-      private org.apache.thrift.protocol.TProtocolFactory protocolFactory;
-      public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) {
-        this.clientManager = clientManager;
-        this.protocolFactory = protocolFactory;
-      }
-      public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) {
-        return new AsyncClient(protocolFactory, clientManager, transport);
-      }
-    }
-
-    public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) {
-      super(protocolFactory, clientManager, transport);
-    }
-
-    public void getServicerAddress(int remoteTableId, org.apache.thrift.async.AsyncMethodCallback<getServicerAddress_call> resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      getServicerAddress_call method_call = new getServicerAddress_call(remoteTableId, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class getServicerAddress_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private int remoteTableId;
-      public getServicerAddress_call(int remoteTableId, org.apache.thrift.async.AsyncMethodCallback<getServicerAddress_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.remoteTableId = remoteTableId;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getServicerAddress", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        getServicerAddress_args args = new getServicerAddress_args();
-        args.setRemoteTableId(remoteTableId);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public String getResult() throws NoServersAvailableException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_getServicerAddress();
-      }
-    }
-
-  }
-
-  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
-    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
-    public Processor(I iface) {
-      super(iface, getProcessMap(new HashMap<String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
-    }
-
-    protected Processor(I iface, Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      super(iface, getProcessMap(processMap));
-    }
-
-    private static <I extends Iface> Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> getProcessMap(Map<String,  org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
-      processMap.put("getServicerAddress", new getServicerAddress());
-      return processMap;
-    }
-
-    public static class getServicerAddress<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getServicerAddress_args> {
-      public getServicerAddress() {
-        super("getServicerAddress");
-      }
-
-      public getServicerAddress_args getEmptyArgsInstance() {
-        return new getServicerAddress_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public getServicerAddress_result getResult(I iface, getServicerAddress_args args) throws org.apache.thrift.TException {
-        getServicerAddress_result result = new getServicerAddress_result();
-        try {
-          result.success = iface.getServicerAddress(args.remoteTableId);
-        } catch (NoServersAvailableException e) {
-          result.e = e;
-        }
-        return result;
-      }
-    }
-
-  }
-
-  public static class getServicerAddress_args implements org.apache.thrift.TBase<getServicerAddress_args, getServicerAddress_args._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getServicerAddress_args");
-
-    private static final org.apache.thrift.protocol.TField REMOTE_TABLE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("remoteTableId", org.apache.thrift.protocol.TType.I32, (short)1);
-
-    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-    static {
-      schemes.put(StandardScheme.class, new getServicerAddress_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new getServicerAddress_argsTupleSchemeFactory());
-    }
-
-    public int remoteTableId; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      REMOTE_TABLE_ID((short)1, "remoteTableId");
-
-      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-      static {
-        for (_Fields field : EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 1: // REMOTE_TABLE_ID
-            return REMOTE_TABLE_ID;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      public static _Fields findByName(String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final String _fieldName;
-
-      _Fields(short thriftId, String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      public String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    private static final int __REMOTETABLEID_ISSET_ID = 0;
-    private byte __isset_bitfield = 0;
-    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.REMOTE_TABLE_ID, new org.apache.thrift.meta_data.FieldMetaData("remoteTableId", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getServicerAddress_args.class, metaDataMap);
-    }
-
-    public getServicerAddress_args() {
-    }
-
-    public getServicerAddress_args(
-      int remoteTableId)
-    {
-      this();
-      this.remoteTableId = remoteTableId;
-      setRemoteTableIdIsSet(true);
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getServicerAddress_args(getServicerAddress_args other) {
-      __isset_bitfield = other.__isset_bitfield;
-      this.remoteTableId = other.remoteTableId;
-    }
-
-    public getServicerAddress_args deepCopy() {
-      return new getServicerAddress_args(this);
-    }
-
-    @Override
-    public void clear() {
-      setRemoteTableIdIsSet(false);
-      this.remoteTableId = 0;
-    }
-
-    public int getRemoteTableId() {
-      return this.remoteTableId;
-    }
-
-    public getServicerAddress_args setRemoteTableId(int remoteTableId) {
-      this.remoteTableId = remoteTableId;
-      setRemoteTableIdIsSet(true);
-      return this;
-    }
-
-    public void unsetRemoteTableId() {
-      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REMOTETABLEID_ISSET_ID);
-    }
-
-    /** Returns true if field remoteTableId is set (has been assigned a value) and false otherwise */
-    public boolean isSetRemoteTableId() {
-      return EncodingUtils.testBit(__isset_bitfield, __REMOTETABLEID_ISSET_ID);
-    }
-
-    public void setRemoteTableIdIsSet(boolean value) {
-      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REMOTETABLEID_ISSET_ID, value);
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        if (value == null) {
-          unsetRemoteTableId();
-        } else {
-          setRemoteTableId((Integer)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        return Integer.valueOf(getRemoteTableId());
-
-      }
-      throw new IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new IllegalArgumentException();
-      }
-
-      switch (field) {
-      case REMOTE_TABLE_ID:
-        return isSetRemoteTableId();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof getServicerAddress_args)
-        return this.equals((getServicerAddress_args)that);
-      return false;
-    }
-
-    public boolean equals(getServicerAddress_args that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_remoteTableId = true;
-      boolean that_present_remoteTableId = true;
-      if (this_present_remoteTableId || that_present_remoteTableId) {
-        if (!(this_present_remoteTableId && that_present_remoteTableId))
-          return false;
-        if (this.remoteTableId != that.remoteTableId)
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      return 0;
-    }
-
-    public int compareTo(getServicerAddress_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-      getServicerAddress_args typedOther = (getServicerAddress_args)other;
-
-      lastComparison = Boolean.valueOf(isSetRemoteTableId()).compareTo(typedOther.isSetRemoteTableId());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRemoteTableId()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.remoteTableId, typedOther.remoteTableId);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder("getServicerAddress_args(");
-      boolean first = true;
-
-      sb.append("remoteTableId:");
-      sb.append(this.remoteTableId);
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-      try {
-        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-        __isset_bitfield = 0;
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getServicerAddress_argsStandardSchemeFactory implements SchemeFactory {
-      public getServicerAddress_argsStandardScheme getScheme() {
-        return new getServicerAddress_argsStandardScheme();
-      }
-    }
-
-    private static class getServicerAddress_argsStandardScheme extends StandardScheme<getServicerAddress_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getServicerAddress_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // REMOTE_TABLE_ID
-              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-                struct.remoteTableId = iprot.readI32();
-                struct.setRemoteTableIdIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked in the validate method
-        struct.validate();
-      }
-
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getServicerAddress_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        oprot.writeFieldBegin(REMOTE_TABLE_ID_FIELD_DESC);
-        oprot.writeI32(struct.remoteTableId);
-        oprot.writeFieldEnd();
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getServicerAddress_argsTupleSchemeFactory implements SchemeFactory {
-      public getServicerAddress_argsTupleScheme getScheme() {
-        return new getServicerAddress_argsTupleScheme();
-      }
-    }
-
-    private static class getServicerAddress_argsTupleScheme extends TupleScheme<getServicerAddress_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetRemoteTableId()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetRemoteTableId()) {
-          oprot.writeI32(struct.remoteTableId);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.remoteTableId = iprot.readI32();
-          struct.setRemoteTableIdIsSet(true);
-        }
-      }
-    }
-
-  }
-
-  public static class getServicerAddress_result implements org.apache.thrift.TBase<getServicerAddress_result, getServicerAddress_result._Fields>, java.io.Serializable, Cloneable   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getServicerAddress_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
-    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-
-    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-    static {
-      schemes.put(StandardScheme.class, new getServicerAddress_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new getServicerAddress_resultTupleSchemeFactory());
-    }
-
-    public String success; // required
-    public NoServersAvailableException e; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    @SuppressWarnings("all") public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success"),
-      E((short)1, "e");
-
-      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-      static {
-        for (_Fields field : EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
-          case 1: // E
-            return E;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      public static _Fields findByName(String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final String _fieldName;
-
-      _Fields(short thriftId, String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      public String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getServicerAddress_result.class, metaDataMap);
-    }
-
-    public getServicerAddress_result() {
-    }
-
-    public getServicerAddress_result(
-      String success,
-      NoServersAvailableException e)
-    {
-      this();
-      this.success = success;
-      this.e = e;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getServicerAddress_result(getServicerAddress_result other) {
-      if (other.isSetSuccess()) {
-        this.success = other.success;
-      }
-      if (other.isSetE()) {
-        this.e = new NoServersAvailableException(other.e);
-      }
-    }
-
-    public getServicerAddress_result deepCopy() {
-      return new getServicerAddress_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.e = null;
-    }
-
-    public String getSuccess() {
-      return this.success;
-    }
-
-    public getServicerAddress_result setSuccess(String success) {
-      this.success = success;
-      return this;
-    }
-
-    public void unsetSuccess() {
-      this.success = null;
-    }
-
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
-    }
-
-    public void setSuccessIsSet(boolean value) {
-      if (!value) {
-        this.success = null;
-      }
-    }
-
-    public NoServersAvailableException getE() {
-      return this.e;
-    }
-
-    public getServicerAddress_result setE(NoServersAvailableException e) {
-      this.e = e;
-      return this;
-    }
-
-    public void unsetE() {
-      this.e = null;
-    }
-
-    /** Returns true if field e is set (has been assigned a value) and false otherwise */
-    public boolean isSetE() {
-      return this.e != null;
-    }
-
-    public void setEIsSet(boolean value) {
-      if (!value) {
-        this.e = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((String)value);
-        }
-        break;
-
-      case E:
-        if (value == null) {
-          unsetE();
-        } else {
-          setE((NoServersAvailableException)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      case E:
-        return getE();
-
-      }
-      throw new IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new IllegalArgumentException();
-      }
-
-      switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
-      case E:
-        return isSetE();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof getServicerAddress_result)
-        return this.equals((getServicerAddress_result)that);
-      return false;
-    }
-
-    public boolean equals(getServicerAddress_result that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
-          return false;
-        if (!this.success.equals(that.success))
-          return false;
-      }
-
-      boolean this_present_e = true && this.isSetE();
-      boolean that_present_e = true && that.isSetE();
-      if (this_present_e || that_present_e) {
-        if (!(this_present_e && that_present_e))
-          return false;
-        if (!this.e.equals(that.e))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      return 0;
-    }
-
-    public int compareTo(getServicerAddress_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-      getServicerAddress_result typedOther = (getServicerAddress_result)other;
-
-      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = Boolean.valueOf(isSetE()).compareTo(typedOther.isSetE());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetE()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, typedOther.e);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-      }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder("getServicerAddress_result(");
-      boolean first = true;
-
-      sb.append("success:");
-      if (this.success == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.success);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("e:");
-      if (this.e == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.e);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getServicerAddress_resultStandardSchemeFactory implements SchemeFactory {
-      public getServicerAddress_resultStandardScheme getScheme() {
-        return new getServicerAddress_resultStandardScheme();
-      }
-    }
-
-    private static class getServicerAddress_resultStandardScheme extends StandardScheme<getServicerAddress_result> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getServicerAddress_result struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.success = iprot.readString();
-                struct.setSuccessIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 1: // E
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.e = new NoServersAvailableException();
-                struct.e.read(iprot);
-                struct.setEIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked in the validate method
-        struct.validate();
-      }
-
-      public void write(org.apache.thrift.protocol.TProtocol oprot, getServicerAddress_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeString(struct.success);
-          oprot.writeFieldEnd();
-        }
-        if (struct.e != null) {
-          oprot.writeFieldBegin(E_FIELD_DESC);
-          struct.e.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getServicerAddress_resultTupleSchemeFactory implements SchemeFactory {
-      public getServicerAddress_resultTupleScheme getScheme() {
-        return new getServicerAddress_resultTupleScheme();
-      }
-    }
-
-    private static class getServicerAddress_resultTupleScheme extends TupleScheme<getServicerAddress_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_result struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetSuccess()) {
-          optionals.set(0);
-        }
-        if (struct.isSetE()) {
-          optionals.set(1);
-        }
-        oprot.writeBitSet(optionals, 2);
-        if (struct.isSetSuccess()) {
-          oprot.writeString(struct.success);
-        }
-        if (struct.isSetE()) {
-          struct.e.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getServicerAddress_result struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(2);
-        if (incoming.get(0)) {
-          struct.success = iprot.readString();
-          struct.setSuccessIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.e = new NoServersAvailableException();
-          struct.e.read(iprot);
-          struct.setEIsSet(true);
-        }
-      }
-    }
-
-  }
-
-}