Blob Blame History Raw
diff --git a/core/src/main/java/org/apache/accumulo/core/rpc/TServiceClientWrapper.java b/core/src/main/java/org/apache/accumulo/core/rpc/TServiceClientWrapper.java
new file mode 100644
index 0000000..9636fe5
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/rpc/TServiceClientWrapper.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.rpc;
+
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.TServiceClient;
+import org.apache.thrift.protocol.TMessage;
+import org.apache.thrift.protocol.TMessageType;
+import org.apache.thrift.protocol.TProtocol;
+
+// Wrapper for THRIFT-4062 workaround; shouldn't be needed in newer versions
+// Also update generate-thrift.sh to stop using this
+public abstract class TServiceClientWrapper extends TServiceClient {
+
+  public TServiceClientWrapper(TProtocol iprot, TProtocol oprot) {
+    super(iprot, oprot);
+  }
+
+  public TServiceClientWrapper(TProtocol prot) {
+    super(prot);
+  }
+
+  @Override
+  protected void receiveBase(TBase<?,?> result, String methodName) throws TException {
+    TMessage msg = iprot_.readMessageBegin();
+    if (msg.type == TMessageType.EXCEPTION) {
+      TApplicationException x = new TApplicationException();
+      x.read(iprot_);
+      iprot_.readMessageEnd();
+      throw x;
+    }
+    if (msg.seqid != seqid_) {
+      throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, String.format("%s failed: out of sequence response: expected %d but got %d",
+          methodName, seqid_, msg.seqid));
+    }
+    result.read(iprot_);
+    iprot_.readMessageEnd();
+  }
+
+}
diff --git a/core/src/main/scripts/generate-thrift.sh b/core/src/main/scripts/generate-thrift.sh
index 691ea79..d189323 100755
--- a/core/src/main/scripts/generate-thrift.sh
+++ b/core/src/main/scripts/generate-thrift.sh
@@ -26,7 +26,7 @@
 #   INCLUDED_MODULES should be an array that includes other Maven modules with src/main/thrift directories
 #   Use INCLUDED_MODULES=(-) in calling scripts that require no other modules
 # ========================================================================================================================
-[[ -z $REQUIRED_THRIFT_VERSION ]] && REQUIRED_THRIFT_VERSION='0.9.3'
+[[ -z $REQUIRED_THRIFT_VERSION ]] && REQUIRED_THRIFT_VERSION='0.10.0'
 [[ -z $INCLUDED_MODULES ]]        && INCLUDED_MODULES=(../server/tracer)
 [[ -z $BASE_OUTPUT_PACKAGE ]]     && BASE_OUTPUT_PACKAGE='org.apache.accumulo.core'
 [[ -z $PACKAGES_TO_GENERATE ]]    && PACKAGES_TO_GENERATE=(gc master tabletserver security client.impl data replication trace)
@@ -71,11 +71,16 @@ for f in src/main/thrift/*.thrift; do
   thrift ${THRIFT_ARGS} --gen cpp "$f" || fail unable to generate cpp thrift classes
 done
 
-# For all generated thrift code, suppress all warnings and add the LICENSE header
-cs='@SuppressWarnings({"unchecked", "serial", "rawtypes", "unused"})'
-es='@SuppressWarnings({"unused"})'
-find $BUILD_DIR/gen-java -name '*.java' -print0 | xargs -0 sed -i.orig -e 's/"unchecked"/"unchecked", "unused"/'
-find $BUILD_DIR/gen-java -name '*.java' -print0 | xargs -0 sed -i.orig -e 's/\(public enum [A-Z]\)/'"$es"' \1/'
+# For all generated thrift code, get rid of all warnings and add the LICENSE header
+
+# workaround for THRIFT-4062; should be fixed in newer thrift versions
+find $BUILD_DIR/gen-java -name '*.java' -exec sed -i -e 's/\(org[.]apache[.]\)thrift\([.]TServiceClient\) /\1accumulo.core.rpc\2Wrapper /' {} +
+# upstream stopped doing import statements for classes, but overlooked enums; delete unused imports
+find $BUILD_DIR/gen-java -name '*.java' -exec grep -Zl '^public enum ' {} + | xargs -0 sed -i -e '/^import .*$/d'
+# add dummy method to suppress "unnecessary suppress warnings" for classes which don't have any unused variables
+# this only affects classes, enums aren't affected
+find $BUILD_DIR/gen-java -name '*.java' -exec grep -Zl '^public class ' {} + | xargs -0 sed -i -e 's/^[}]$/  private static void unusedMethod() {}\
+}/'
 
 for lang in "${LANGUAGES_TO_GENERATE[@]}"; do
   case $lang in
diff --git a/core/src/main/thrift/client.thrift b/core/src/main/thrift/client.thrift
index 38a8076..2268574 100644
--- a/core/src/main/thrift/client.thrift
+++ b/core/src/main/thrift/client.thrift
@@ -129,15 +129,15 @@ service ClientService {
   list<binary> getUserAuthorizations(3:trace.TInfo tinfo, 4:security.TCredentials credentials, 2:string principal) throws (1:ThriftSecurityException sec)
 
   // permissions-related methods
-  bool hasSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte sysPerm) throws (1:ThriftSecurityException sec)
-  bool hasTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte tblPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-  bool hasNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte tblNspcPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-  void grantSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
-  void revokeSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:byte permission) throws (1:ThriftSecurityException sec)
-  void grantTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-  void revokeTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-  void grantNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
-  void revokeNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:byte permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  bool hasSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:i8 sysPerm) throws (1:ThriftSecurityException sec)
+  bool hasTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:i8 tblPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  bool hasNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:i8 tblNspcPerm) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  void grantSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:i8 permission) throws (1:ThriftSecurityException sec)
+  void revokeSystemPermission(4:trace.TInfo tinfo, 5:security.TCredentials credentials, 2:string principal, 3:i8 permission) throws (1:ThriftSecurityException sec)
+  void grantTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:i8 permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  void revokeTablePermission(5:trace.TInfo tinfo, 6:security.TCredentials credentials, 2:string principal, 3:string tableName, 4:i8 permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  void grantNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:i8 permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
+  void revokeNamespacePermission(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:string principal, 4:string ns, 5:i8 permission) throws (1:ThriftSecurityException sec, 2:ThriftTableOperationException tope)
 
   // configuration methods
   map<string, string> getConfiguration(2:trace.TInfo tinfo, 3:security.TCredentials credentials, 1:ConfigurationType type);
diff --git a/core/src/main/thrift/master.thrift b/core/src/main/thrift/master.thrift
index b959074..832c738 100644
--- a/core/src/main/thrift/master.thrift
+++ b/core/src/main/thrift/master.thrift
@@ -113,7 +113,7 @@ struct DeadServer {
 struct MasterMonitorInfo {
   1:map<string, TableInfo> tableMap
   2:list<TabletServerStatus> tServerInfo
-  3:map<string, byte> badTServers
+  3:map<string, i8> badTServers
   6:MasterState state
   8:MasterGoalState goalState
   7:i32 unassignedTablets
diff --git a/pom.xml b/pom.xml
index aea9399..b804ada 100644
--- a/pom.xml
+++ b/pom.xml
@@ -149,7 +149,7 @@
     <surefire.failIfNoSpecifiedTests>false</surefire.failIfNoSpecifiedTests>
     <surefire.groups />
     <!-- Thrift version -->
-    <thrift.version>0.9.3</thrift.version>
+    <thrift.version>0.10.0</thrift.version>
     <!-- ZooKeeper version -->
     <zookeeper.version>3.4.6</zookeeper.version>
   </properties>
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java b/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
index 89c755d..5ab5263 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/Proxy.java
@@ -212,8 +212,7 @@ public class Proxy implements KeywordExecutable {
     ProxyServer impl = new ProxyServer(properties);
 
     // Wrap the implementation -- translate some exceptions
-    AccumuloProxy.Iface wrappedImpl = RpcWrapper.service(impl,
-        new AccumuloProxy.Processor<AccumuloProxy.Iface>(impl));
+    AccumuloProxy.Iface wrappedImpl = RpcWrapper.service(impl);
 
     // Create the processor from the implementation
     TProcessor processor = new AccumuloProxy.Processor<>(wrappedImpl);
diff --git a/server/base/src/main/java/org/apache/accumulo/server/rpc/RpcWrapper.java b/server/base/src/main/java/org/apache/accumulo/server/rpc/RpcWrapper.java
index b9df911..3e9d8d5 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/rpc/RpcWrapper.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/rpc/RpcWrapper.java
@@ -19,19 +19,11 @@ package org.apache.accumulo.server.rpc;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
 
 import org.apache.accumulo.core.trace.wrappers.RpcServerInvocationHandler;
 import org.apache.accumulo.core.trace.wrappers.TraceWrap;
-import org.apache.thrift.ProcessFunction;
 import org.apache.thrift.TApplicationException;
-import org.apache.thrift.TBaseProcessor;
 import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * This class accommodates the changes in THRIFT-1805, which appeared in Thrift 0.9.1 and restricts
@@ -56,14 +48,9 @@ import org.slf4j.LoggerFactory;
  * @since 1.6.1
  */
 public class RpcWrapper {
-  private static final Logger log = LoggerFactory.getLogger(RpcWrapper.class);
 
-  public static <I> I service(final I instance, final TBaseProcessor<I> processor) {
-    final Map<String,ProcessFunction<I,?>> processorView = processor.getProcessMapView();
-    final Set<String> onewayMethods = getOnewayMethods(processorView);
-    log.debug("Found oneway Thrift methods: " + onewayMethods);
-
-    InvocationHandler handler = getInvocationHandler(instance, onewayMethods);
+  public static <I> I service(final I instance) {
+    InvocationHandler handler = getInvocationHandler(instance);
 
     @SuppressWarnings("unchecked")
     I proxiedInstance = (I) Proxy.newProxyInstance(instance.getClass().getClassLoader(),
@@ -71,10 +58,8 @@ public class RpcWrapper {
     return proxiedInstance;
   }
 
-  protected static <T> RpcServerInvocationHandler<T> getInvocationHandler(final T instance,
-      final Set<String> onewayMethods) {
+  protected static <T> RpcServerInvocationHandler<T> getInvocationHandler(final T instance) {
     return new RpcServerInvocationHandler<T>(instance) {
-      private final Logger log = LoggerFactory.getLogger(instance.getClass());
 
       @Override
       public Object invoke(Object obj, Method method, Object[] args) throws Throwable {
@@ -82,57 +67,10 @@ public class RpcWrapper {
         try {
           return super.invoke(obj, method, args);
         } catch (RuntimeException e) {
-          String msg = e.getMessage();
-          log.error(msg, e);
-          if (onewayMethods.contains(method.getName())) {
-            throw e;
-          }
-          throw new TException(msg);
-        } catch (Error e) {
-          String msg = e.getMessage();
-          log.error(msg, e);
-          if (onewayMethods.contains(method.getName())) {
-            throw e;
-          }
-          throw new TException(msg);
+          // thrift will log the exception in ProcessFunction
+          throw new TException(e);
         }
       }
     };
   }
-
-  protected static Set<String> getOnewayMethods(Map<String,?> processorView) {
-    // Get a handle on the isOnewayMethod and make it accessible
-    final Method isOnewayMethod;
-    try {
-      isOnewayMethod = ProcessFunction.class.getDeclaredMethod("isOneway");
-    } catch (NoSuchMethodException e) {
-      throw new RuntimeException("Could not access isOneway method", e);
-    } catch (SecurityException e) {
-      throw new RuntimeException("Could not access isOneway method", e);
-    }
-    // In java7, this appears to be copying the method, but it's trivial for us to return the object
-    // to how it was before.
-    final boolean accessible = isOnewayMethod.isAccessible();
-    isOnewayMethod.setAccessible(true);
-
-    try {
-      final Set<String> onewayMethods = new HashSet<>();
-      for (Entry<String,?> entry : processorView.entrySet()) {
-        try {
-          if ((Boolean) isOnewayMethod.invoke(entry.getValue())) {
-            onewayMethods.add(entry.getKey());
-          }
-        } catch (RuntimeException e) {
-          throw e;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-
-      return onewayMethods;
-    } finally {
-      // Reset it back to how it was.
-      isOnewayMethod.setAccessible(accessible);
-    }
-  }
 }
diff --git a/server/base/src/test/java/org/apache/accumulo/server/rpc/RpcWrapperTest.java b/server/base/src/test/java/org/apache/accumulo/server/rpc/RpcWrapperTest.java
deleted file mode 100644
index a942325..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/rpc/RpcWrapperTest.java
+++ /dev/null
@@ -1,307 +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.
- */
-package org.apache.accumulo.server.rpc;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.accumulo.core.trace.wrappers.RpcServerInvocationHandler;
-import org.apache.thrift.ProcessFunction;
-import org.apache.thrift.TBase;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TProtocol;
-import org.junit.Assert;
-import org.junit.Test;
-
-import com.google.common.collect.Sets;
-
-/**
- * Verification that RpcWrapper correctly mangles Exceptions to work around Thrift.
- */
-public class RpcWrapperTest {
-
-  private static final String RTE_MESSAGE = "RpcWrapperTest's RuntimeException Message";
-
-  /**
-   * Given a method name and whether or not the method is oneway, construct a ProcessFunction.
-   *
-   * @param methodName
-   *          The service method name.
-   * @param isOneway
-   *          Is the method oneway.
-   * @return A ProcessFunction.
-   */
-  private fake_proc<FakeService> createProcessFunction(String methodName, boolean isOneway) {
-    return new fake_proc<>(methodName, isOneway);
-  }
-
-  @Test
-  public void testSomeOnewayMethods() {
-    Map<String,ProcessFunction<FakeService,?>> procs = new HashMap<>();
-    procs.put("foo", createProcessFunction("foo", true));
-    procs.put("foobar", createProcessFunction("foobar", false));
-    procs.put("bar", createProcessFunction("bar", true));
-    procs.put("barfoo", createProcessFunction("barfoo", false));
-
-    Set<String> onewayMethods = RpcWrapper.getOnewayMethods(procs);
-    Assert.assertEquals(Sets.newHashSet("foo", "bar"), onewayMethods);
-  }
-
-  @Test
-  public void testNoOnewayMethods() {
-    Map<String,ProcessFunction<FakeService,?>> procs = new HashMap<>();
-    procs.put("foo", createProcessFunction("foo", false));
-    procs.put("foobar", createProcessFunction("foobar", false));
-    procs.put("bar", createProcessFunction("bar", false));
-    procs.put("barfoo", createProcessFunction("barfoo", false));
-
-    Set<String> onewayMethods = RpcWrapper.getOnewayMethods(procs);
-    Assert.assertEquals(Collections.<String> emptySet(), onewayMethods);
-  }
-
-  @Test
-  public void testAllOnewayMethods() {
-    Map<String,ProcessFunction<FakeService,?>> procs = new HashMap<>();
-    procs.put("foo", createProcessFunction("foo", true));
-    procs.put("foobar", createProcessFunction("foobar", true));
-    procs.put("bar", createProcessFunction("bar", true));
-    procs.put("barfoo", createProcessFunction("barfoo", true));
-
-    Set<String> onewayMethods = RpcWrapper.getOnewayMethods(procs);
-    Assert.assertEquals(Sets.newHashSet("foo", "foobar", "bar", "barfoo"), onewayMethods);
-  }
-
-  @Test
-  public void testNoExceptionWrappingForOneway() throws Throwable {
-    final Object[] args = new Object[0];
-
-    final FakeService impl = new FakeServiceImpl();
-
-    // "short" names throw RTEs and are oneway, while long names do not throw exceptions and are not
-    // oneway.
-    RpcServerInvocationHandler<FakeService> handler = RpcWrapper.getInvocationHandler(impl,
-        Sets.newHashSet("foo", "bar"));
-
-    // Should throw an exception, but not be wrapped because the method is oneway
-    try {
-      handler.invoke(impl, FakeServiceImpl.class.getMethod("foo"), args);
-      Assert.fail("Expected an exception");
-    } catch (RuntimeException e) {
-      Assert.assertEquals(RTE_MESSAGE, e.getMessage());
-    }
-
-    // Should not throw an exception
-    handler.invoke(impl, FakeServiceImpl.class.getMethod("foobar"), args);
-  }
-
-  @Test
-  public void testExceptionWrappingForNonOneway() throws Throwable {
-    final Object[] args = new Object[0];
-
-    final FakeService impl = new FakeServiceImpl();
-
-    // "short" names throw RTEs and are not oneway, while long names do not throw exceptions and are
-    // oneway.
-    RpcServerInvocationHandler<FakeService> handler = RpcWrapper.getInvocationHandler(impl,
-        Sets.newHashSet("foobar", "barfoo"));
-
-    // Should throw an exception, but not be wrapped because the method is oneway
-    try {
-      handler.invoke(impl, FakeServiceImpl.class.getMethod("foo"), args);
-      Assert.fail("Expected an exception");
-    } catch (TException e) {
-      // The InvocationHandler should take the exception from the RTE and make it a TException
-      Assert.assertEquals(RTE_MESSAGE, e.getMessage());
-    }
-
-    // Should not throw an exception
-    handler.invoke(impl, FakeServiceImpl.class.getMethod("foobar"), args);
-  }
-
-  //
-  // Some hacked together classes/interfaces that mimic what Thrift is doing.
-  //
-
-  /**
-   * Some fake fields for our fake arguments.
-   */
-  private static class fake_fields implements org.apache.thrift.TFieldIdEnum {
-    @Override
-    public short getThriftFieldId() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public String getFieldName() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public int hashCode() {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  /**
-   * A fake thrift service
-   */
-  interface FakeService {
-    void foo();
-
-    String foobar();
-
-    int bar();
-
-    long barfoo();
-  }
-
-  /**
-   * An implementation of the fake thrift service. The "short" names throw RTEs, while long names do
-   * not.
-   */
-  public static class FakeServiceImpl implements FakeService {
-    @Override
-    public void foo() {
-      throw new RuntimeException(RTE_MESSAGE);
-    }
-
-    @Override
-    public String foobar() {
-      return "";
-    }
-
-    @Override
-    public int bar() {
-      throw new RuntimeException(RTE_MESSAGE);
-    }
-
-    @Override
-    public long barfoo() {
-      return 0;
-    }
-  }
-
-  /**
-   * A fake ProcessFunction implementation for testing that allows injection of method name and
-   * oneway.
-   */
-  private static class fake_proc<I extends FakeService>
-      extends org.apache.thrift.ProcessFunction<I,foo_args> {
-    final private boolean isOneway;
-
-    public fake_proc(String methodName, boolean isOneway) {
-      super(methodName);
-      this.isOneway = isOneway;
-    }
-
-    @Override
-    protected boolean isOneway() {
-      return isOneway;
-    }
-
-    @SuppressWarnings("rawtypes")
-    @Override
-    public TBase getResult(I iface, foo_args args) throws TException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public foo_args getEmptyArgsInstance() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public int hashCode() {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  /**
-   * Fake arguments for our fake service.
-   */
-  private static class foo_args implements org.apache.thrift.TBase<foo_args,fake_fields> {
-
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public boolean equals(Object o) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public int hashCode() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public int compareTo(foo_args o) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void read(TProtocol iprot) throws TException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void write(TProtocol oprot) throws TException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public fake_fields fieldForId(int fieldId) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean isSet(fake_fields field) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public Object getFieldValue(fake_fields field) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void setFieldValue(fake_fields field, Object value) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public TBase<foo_args,fake_fields> deepCopy() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void clear() {
-      throw new UnsupportedOperationException();
-    }
-  }
-}
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
index ab6fa88..18d9377 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
@@ -354,7 +354,7 @@ public class TServerUtilsTest {
   private ServerAddress startServer() throws Exception {
     AccumuloServerContext ctx = new AccumuloServerContext(factory);
     ClientServiceHandler clientHandler = new ClientServiceHandler(ctx, null, null);
-    Iface rpcProxy = RpcWrapper.service(clientHandler, new Processor<Iface>(clientHandler));
+    Iface rpcProxy = RpcWrapper.service(clientHandler);
     Processor<Iface> processor = new Processor<>(rpcProxy);
     // "localhost" explicitly to make sure we can always bind to that interface (avoids DNS
     // misconfiguration)
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 0aeaaea..62f63f0 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
@@ -751,7 +751,7 @@ public class SimpleGarbageCollector extends AccumuloServerContext implements Ifa
   }
 
   private HostAndPort startStatsService() throws UnknownHostException {
-    Iface rpcProxy = RpcWrapper.service(this, new Processor<Iface>(this));
+    Iface rpcProxy = RpcWrapper.service(this);
     final Processor<Iface> processor;
     if (ThriftServerType.SASL == getThriftServerType()) {
       Iface tcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, getClass(), getConfiguration());
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index 9414b98..bac30b1 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -1318,7 +1318,7 @@ public class Master extends AccumuloServerContext
     }
 
     clientHandler = new MasterClientServiceHandler(this);
-    Iface rpcProxy = RpcWrapper.service(clientHandler, new Processor<Iface>(clientHandler));
+    Iface rpcProxy = RpcWrapper.service(clientHandler);
     final Processor<Iface> processor;
     if (ThriftServerType.SASL == getThriftServerType()) {
       Iface tcredsProxy = TCredentialsUpdatingWrapper.service(rpcProxy, clientHandler.getClass(),
@@ -1358,8 +1358,7 @@ public class Master extends AccumuloServerContext
     ReplicationCoordinator.Processor<ReplicationCoordinator.Iface> replicationCoordinatorProcessor =
       new ReplicationCoordinator.Processor<>(
     // @formatter:on
-            RpcWrapper.service(impl,
-                new ReplicationCoordinator.Processor<ReplicationCoordinator.Iface>(impl)));
+            RpcWrapper.service((ReplicationCoordinator.Iface) impl));
     ServerAddress replAddress = TServerUtils.startServer(this, hostname,
         Property.MASTER_REPLICATION_COORDINATOR_PORT, replicationCoordinatorProcessor,
         "Master Replication Coordinator", "Replication Coordinator", null,
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index f87841d..6b5062c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -2508,7 +2508,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   private HostAndPort startTabletClientService() throws UnknownHostException {
     // start listening for client connection last
     clientHandler = new ThriftClientHandler();
-    Iface rpcProxy = RpcWrapper.service(clientHandler, new Processor<Iface>(clientHandler));
+    Iface rpcProxy = RpcWrapper.service(clientHandler);
     final Processor<Iface> processor;
     if (ThriftServerType.SASL == getThriftServerType()) {
       Iface tcredProxy = TCredentialsUpdatingWrapper.service(rpcProxy, ThriftClientHandler.class,
@@ -2525,8 +2525,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
 
   private HostAndPort startReplicationService() throws UnknownHostException {
     final ReplicationServicerHandler handler = new ReplicationServicerHandler(this);
-    ReplicationServicer.Iface rpcProxy = RpcWrapper.service(handler,
-        new ReplicationServicer.Processor<ReplicationServicer.Iface>(handler));
+    ReplicationServicer.Iface rpcProxy = RpcWrapper.service(handler);
     ReplicationServicer.Iface repl = TCredentialsUpdatingWrapper.service(rpcProxy,
         handler.getClass(), getConfiguration());
     // @formatter:off
diff --git a/test/pom.xml b/test/pom.xml
index f6a9d00..7ef55c4 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -277,6 +277,29 @@
     </plugins>
   </build>
   <profiles>
+    <profile>
+      <id>thrift</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>exec-maven-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>generate-thrift</id>
+                <goals>
+                  <goal>exec</goal>
+                </goals>
+                <phase>generate-sources</phase>
+                <configuration>
+                  <executable>${basedir}/src/main/scripts/generate-thrift.sh</executable>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
     <profile>
       <!-- create shaded test jar appropriate for running ITs on MapReduce -->
       <id>mrit</id>
diff --git a/test/src/main/findbugs/exclude-filter.xml b/test/src/main/findbugs/exclude-filter.xml
index 3c3e596..9339c36 100644
--- a/test/src/main/findbugs/exclude-filter.xml
+++ b/test/src/main/findbugs/exclude-filter.xml
@@ -15,6 +15,12 @@
   limitations under the License.
 -->
 <FindBugsFilter>
+  <Match>
+    <!-- ignore thrift-generated files -->
+    <Or>
+      <Package name="org.apache.accumulo.test.rpc.thrift" />
+    </Or>
+  </Match>
   <Match>
     <!-- ignore intentional infinite loop in test main methods -->
     <Or>
diff --git a/test/src/main/java/org/apache/accumulo/test/rpc/Mocket.java b/test/src/main/java/org/apache/accumulo/test/rpc/Mocket.java
new file mode 100644
index 0000000..81c774d
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/rpc/Mocket.java
@@ -0,0 +1,194 @@
+/*
+ * 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.test.rpc;
+
+import java.util.Objects;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.thrift.transport.TServerTransport;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Mocket - a Mock Socket
+ * <p>
+ * Implements a bi-directional client-server transport in memory, using two FIFO queues. The output stream of the client is wired to the input stream of the
+ * server, and the output stream of the server is wired to the input stream of the client.
+ */
+public class Mocket {
+
+  private final TTransport clientTransport;
+  private final TServerTransport serverTransport;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
+
+  public Mocket() {
+    Buffer serverQueue = new Buffer();
+    Buffer clientQueue = new Buffer();
+    // wire up the two queues to each other
+    clientTransport = new MocketTransport(clientQueue, serverQueue);
+    serverTransport = new MocketServerTransport(new MocketTransport(serverQueue, clientQueue));
+
+  }
+
+  public TServerTransport getServerTransport() {
+    return serverTransport;
+  }
+
+  public TTransport getClientTransport() {
+    return clientTransport;
+  }
+
+  private boolean isMocketClosed() {
+    return closed.get();
+  }
+
+  private void closeMocket() {
+    closed.set(true);
+  }
+
+  private class Buffer {
+
+    private final LinkedBlockingQueue<Integer> queue = new LinkedBlockingQueue<>();
+
+    public void write(int b) {
+      queue.add(b);
+    }
+
+    public void write(byte[] buf, int off, int len) {
+      Objects.requireNonNull(buf);
+      Preconditions.checkPositionIndexes(off, off + len, buf.length);
+      if (len == 0) {
+        return;
+      }
+      for (int i = 0; i < len; i++) {
+        write(buf[off + i]);
+      }
+    }
+
+    public int read() {
+      Integer item;
+      // item = queue.take();
+      // loop below makes sure we don't block indefinitely
+      while (!isMocketClosed()) {
+        try {
+          item = queue.poll(10, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException e) {
+          // reset interrupt flag before returning
+          Thread.currentThread().interrupt();
+          closeMocket();
+          return -1;
+        }
+        // null means the timeout was reached
+        if (item != null) {
+          return item;
+        }
+      }
+      return -1;
+    }
+
+    public int read(byte[] buf, int off, int len) {
+      Objects.requireNonNull(buf);
+      Preconditions.checkPositionIndexes(off, off + len, buf.length);
+      if (len == 0) {
+        return 0;
+      }
+      int c = read();
+      if (c == -1) {
+        return -1;
+      }
+      buf[off] = (byte) c;
+
+      int i;
+      for (i = 1; i < len; i++) {
+        c = read();
+        if (c == -1) {
+          break;
+        }
+        buf[off + i] = (byte) c;
+      }
+      return i;
+    }
+
+  }
+
+  private class MocketServerTransport extends TServerTransport {
+
+    private final MocketTransport servTrans;
+
+    public MocketServerTransport(MocketTransport mocketTransport) {
+      servTrans = mocketTransport;
+    }
+
+    @Override
+    public void listen() throws TTransportException {}
+
+    @Override
+    public void close() {
+      acceptImpl().close();
+    }
+
+    @Override
+    protected TTransport acceptImpl() {
+      return servTrans;
+    }
+
+    @Override
+    public void interrupt() {
+      close();
+    }
+
+  }
+
+  private class MocketTransport extends TTransport {
+
+    private final Buffer input;
+    private final Buffer output;
+
+    private MocketTransport(Buffer input, Buffer output) {
+      this.input = input;
+      this.output = output;
+    }
+
+    @Override
+    public void write(byte[] buf, int off, int len) throws TTransportException {
+      output.write(buf, off, len);
+    }
+
+    @Override
+    public int read(byte[] buf, int off, int len) throws TTransportException {
+      return input.read(buf, off, len);
+    }
+
+    @Override
+    public void open() throws TTransportException {}
+
+    @Override
+    public boolean isOpen() {
+      return !isMocketClosed();
+    }
+
+    @Override
+    public void close() {
+      closeMocket();
+    }
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/rpc/SimpleThriftServiceHandler.java b/test/src/main/java/org/apache/accumulo/test/rpc/SimpleThriftServiceHandler.java
new file mode 100644
index 0000000..02db55c
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/rpc/SimpleThriftServiceHandler.java
@@ -0,0 +1,63 @@
+/*
+ * 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.test.rpc;
+
+import org.apache.accumulo.test.rpc.thrift.SimpleThriftService;
+import org.apache.thrift.TException;
+
+public class SimpleThriftServiceHandler implements SimpleThriftService.Iface {
+
+  private void setProp(String method, String value) {
+    System.setProperty(this.getClass().getSimpleName() + "." + method, value);
+  }
+
+  @Override
+  public String echoFail(String value) throws TException {
+    setProp("echoFail", value);
+    throw new TException(new UnsupportedOperationException(value));
+  }
+
+  @Override
+  public String echoRuntimeFail(String value) {
+    setProp("echoRuntimeFail", value);
+    throw new UnsupportedOperationException(value);
+  }
+
+  @Override
+  public String echoPass(String value) {
+    setProp("echoPass", value);
+    return value;
+  }
+
+  @Override
+  public void onewayFail(String value) throws TException {
+    setProp("onewayFail", value);
+    throw new TException(new UnsupportedOperationException(value));
+  }
+
+  @Override
+  public void onewayRuntimeFail(String value) {
+    setProp("onewayRuntimeFail", value);
+    throw new UnsupportedOperationException(value);
+  }
+
+  @Override
+  public void onewayPass(String value) {
+    setProp("onewayPass", value);
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/rpc/SimpleThriftServiceRunner.java b/test/src/main/java/org/apache/accumulo/test/rpc/SimpleThriftServiceRunner.java
new file mode 100644
index 0000000..f2865c2
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/rpc/SimpleThriftServiceRunner.java
@@ -0,0 +1,75 @@
+/*
+ * 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.test.rpc;
+
+import org.apache.accumulo.server.rpc.RpcWrapper;
+import org.apache.accumulo.test.rpc.thrift.SimpleThriftService;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.server.TSimpleServer;
+
+/**
+ * A utility for starting a simple thrift server, and providing a corresponding client for in-memory testing of thrift behavior
+ */
+public class SimpleThriftServiceRunner {
+
+  private SimpleThriftServiceHandler handler = new SimpleThriftServiceHandler();
+
+  private final Mocket mocket;
+  private final Thread serviceThread;
+  private final TServer server;
+
+  public SimpleThriftServiceRunner(String threadName, boolean useWrapper) {
+    this.mocket = new Mocket();
+    this.server = createServer(useWrapper);
+    this.serviceThread = new Thread(() -> server.serve(), threadName);
+  }
+
+  public void startService() {
+    serviceThread.start();
+  }
+
+  public SimpleThriftServiceHandler handler() {
+    return handler;
+  }
+
+  public SimpleThriftService.Client client() {
+    return new SimpleThriftService.Client(new TBinaryProtocol(mocket.getClientTransport()));
+  }
+
+  private TServer createServer(boolean useWrapper) {
+    TServer.Args args = new TServer.Args(mocket.getServerTransport());
+    SimpleThriftService.Iface actualHandler = handler;
+    if (useWrapper) {
+      actualHandler = RpcWrapper.<SimpleThriftService.Iface> service(handler);
+    }
+    args.processor(new SimpleThriftService.Processor<>(actualHandler));
+    args.protocolFactory(new TBinaryProtocol.Factory());
+    return new TSimpleServer(args);
+  }
+
+  public void stopService() {
+    server.stop();
+    try {
+      serviceThread.join();
+    } catch (InterruptedException e) {
+      // re-set interrupt flag
+      Thread.currentThread().interrupt();
+    }
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/rpc/ThriftBehaviorIT.java b/test/src/main/java/org/apache/accumulo/test/rpc/ThriftBehaviorIT.java
new file mode 100644
index 0000000..ee6fa08
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/rpc/ThriftBehaviorIT.java
@@ -0,0 +1,193 @@
+/*
+ * 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.test.rpc;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.test.categories.SunnyDayTests;
+import org.apache.accumulo.test.rpc.thrift.SimpleThriftService;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.thrift.ProcessFunction;
+import org.apache.thrift.TApplicationException;
+import org.apache.thrift.TException;
+import org.apache.thrift.server.TSimpleServer;
+import org.hamcrest.core.IsInstanceOf;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
+
+@Category(SunnyDayTests.class)
+public class ThriftBehaviorIT {
+
+  @Rule
+  public Timeout timeout = new Timeout(5, TimeUnit.SECONDS);
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  private SimpleThriftService.Client client;
+  private SimpleThriftServiceHandler handler;
+  private SimpleThriftServiceRunner serviceRunner;
+  private String propName;
+  private Map<Logger,Level> oldLogLevels = new HashMap<>();
+
+  private static final String KITTY_MSG = "🐈 Kitty! 🐈";
+
+  // can delete wrapper when tests pass without using it (assuming tests are good enough)
+  private static final boolean USE_RPC_WRAPPER = true;
+
+  private static final boolean SUPPRESS_SPAMMY_LOGGERS = true;
+
+  @Before
+  public void createClientAndServer() {
+    Arrays.stream(new Class<?>[] {TSimpleServer.class, ProcessFunction.class}).forEach(spammyClass -> {
+      Logger spammyLogger = Logger.getLogger(spammyClass);
+      oldLogLevels.put(spammyLogger, spammyLogger.getLevel());
+      if (SUPPRESS_SPAMMY_LOGGERS) {
+        spammyLogger.setLevel(Level.OFF);
+      }
+    });
+
+    String threadName = ThriftBehaviorIT.class.getSimpleName() + "." + testName.getMethodName();
+    serviceRunner = new SimpleThriftServiceRunner(threadName, USE_RPC_WRAPPER);
+    serviceRunner.startService();
+    client = serviceRunner.client();
+    handler = serviceRunner.handler();
+
+    propName = testName.getMethodName();
+    if (propName.endsWith("Handler")) {
+      propName = propName.substring(0, propName.length() - 7);
+    }
+    propName = SimpleThriftServiceHandler.class.getSimpleName() + "." + propName;
+
+    // make sure the property is reset before the test runs
+    System.setProperty(propName, "-");
+    Assert.assertEquals("-", System.getProperty(propName));
+  }
+
+  @After
+  public void shutdownServer() {
+    serviceRunner.stopService();
+
+    oldLogLevels.forEach((spammyLogger, oldLevel) -> {
+      spammyLogger.setLevel(oldLevel);
+    });
+
+    // make sure the method was actually executed by the service handler
+    Assert.assertEquals(KITTY_MSG, System.getProperty(propName));
+  }
+
+  @Test
+  public void echoFailHandler() throws TException {
+    exception.expect(TException.class);
+    exception.expectCause(IsInstanceOf.instanceOf(UnsupportedOperationException.class));
+    handler.echoFail(KITTY_MSG);
+  }
+
+  @Test
+  public void echoFail() throws TException {
+    try {
+      client.echoFail(KITTY_MSG);
+      Assert.fail("Thrift client did not throw an expected exception");
+    } catch (Exception e) {
+      Assert.assertEquals(TApplicationException.class.getName(), e.getClass().getName());
+    }
+    // verify normal two-way method still passes using same client
+    echoPass();
+  }
+
+  @Test
+  public void echoRuntimeFailHandler() throws TException {
+    exception.expect(UnsupportedOperationException.class);
+    handler.echoRuntimeFail(KITTY_MSG);
+  }
+
+  @Test
+  public void echoRuntimeFail() throws TException {
+    try {
+      client.echoRuntimeFail(KITTY_MSG);
+      Assert.fail("Thrift client did not throw an expected exception");
+    } catch (Exception e) {
+      Assert.assertEquals(TApplicationException.class.getName(), e.getClass().getName());
+    }
+    // verify normal two-way method still passes using same client
+    echoPass();
+  }
+
+  @Test
+  public void echoPassHandler() {
+    Assert.assertEquals(KITTY_MSG, handler.echoPass(KITTY_MSG));
+  }
+
+  @Test
+  public void echoPass() throws TException {
+    Assert.assertEquals(KITTY_MSG, client.echoPass(KITTY_MSG));
+  }
+
+  @Test
+  public void onewayFailHandler() throws TException {
+    exception.expect(TException.class);
+    exception.expectCause(IsInstanceOf.instanceOf(UnsupportedOperationException.class));
+    handler.onewayFail(KITTY_MSG);
+  }
+
+  @Test
+  public void onewayFail() throws TException {
+    client.onewayFail(KITTY_MSG);
+    // verify normal two-way method still passes using same client
+    echoPass();
+  }
+
+  @Test
+  public void onewayRuntimeFailHandler() throws TException {
+    exception.expect(UnsupportedOperationException.class);
+    handler.onewayRuntimeFail(KITTY_MSG);
+  }
+
+  @Test
+  public void onewayRuntimeFail() throws TException {
+    client.onewayRuntimeFail(KITTY_MSG);
+    // verify normal two-way method still passes using same client
+    echoPass();
+  }
+
+  @Test
+  public void onewayPassHandler() {
+    handler.onewayPass(KITTY_MSG);
+  }
+
+  @Test
+  public void onewayPass() throws TException {
+    client.onewayPass(KITTY_MSG);
+    // verify normal two-way method still passes using same client
+    echoPass();
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/rpc/thrift/SimpleThriftService.java b/test/src/main/java/org/apache/accumulo/test/rpc/thrift/SimpleThriftService.java
new file mode 100644
index 0000000..d76a789
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/rpc/thrift/SimpleThriftService.java
@@ -0,0 +1,4103 @@
+/*
+ * 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.10.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.accumulo.test.rpc.thrift;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.10.0)")
+public class SimpleThriftService {
+
+  public interface Iface {
+
+    public java.lang.String echoPass(java.lang.String value) throws org.apache.thrift.TException;
+
+    public void onewayPass(java.lang.String value) throws org.apache.thrift.TException;
+
+    public java.lang.String echoFail(java.lang.String value) throws org.apache.thrift.TException;
+
+    public void onewayFail(java.lang.String value) throws org.apache.thrift.TException;
+
+    public java.lang.String echoRuntimeFail(java.lang.String value) throws org.apache.thrift.TException;
+
+    public void onewayRuntimeFail(java.lang.String value) throws org.apache.thrift.TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void echoPass(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+
+    public void onewayPass(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void echoFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+
+    public void onewayFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+    public void echoRuntimeFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException;
+
+    public void onewayRuntimeFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException;
+
+  }
+
+  public static class Client extends org.apache.accumulo.core.rpc.TServiceClientWrapper 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 java.lang.String echoPass(java.lang.String value) throws org.apache.thrift.TException
+    {
+      send_echoPass(value);
+      return recv_echoPass();
+    }
+
+    public void send_echoPass(java.lang.String value) throws org.apache.thrift.TException
+    {
+      echoPass_args args = new echoPass_args();
+      args.setValue(value);
+      sendBase("echoPass", args);
+    }
+
+    public java.lang.String recv_echoPass() throws org.apache.thrift.TException
+    {
+      echoPass_result result = new echoPass_result();
+      receiveBase(result, "echoPass");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "echoPass failed: unknown result");
+    }
+
+    public void onewayPass(java.lang.String value) throws org.apache.thrift.TException
+    {
+      send_onewayPass(value);
+    }
+
+    public void send_onewayPass(java.lang.String value) throws org.apache.thrift.TException
+    {
+      onewayPass_args args = new onewayPass_args();
+      args.setValue(value);
+      sendBaseOneway("onewayPass", args);
+    }
+
+    public java.lang.String echoFail(java.lang.String value) throws org.apache.thrift.TException
+    {
+      send_echoFail(value);
+      return recv_echoFail();
+    }
+
+    public void send_echoFail(java.lang.String value) throws org.apache.thrift.TException
+    {
+      echoFail_args args = new echoFail_args();
+      args.setValue(value);
+      sendBase("echoFail", args);
+    }
+
+    public java.lang.String recv_echoFail() throws org.apache.thrift.TException
+    {
+      echoFail_result result = new echoFail_result();
+      receiveBase(result, "echoFail");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "echoFail failed: unknown result");
+    }
+
+    public void onewayFail(java.lang.String value) throws org.apache.thrift.TException
+    {
+      send_onewayFail(value);
+    }
+
+    public void send_onewayFail(java.lang.String value) throws org.apache.thrift.TException
+    {
+      onewayFail_args args = new onewayFail_args();
+      args.setValue(value);
+      sendBaseOneway("onewayFail", args);
+    }
+
+    public java.lang.String echoRuntimeFail(java.lang.String value) throws org.apache.thrift.TException
+    {
+      send_echoRuntimeFail(value);
+      return recv_echoRuntimeFail();
+    }
+
+    public void send_echoRuntimeFail(java.lang.String value) throws org.apache.thrift.TException
+    {
+      echoRuntimeFail_args args = new echoRuntimeFail_args();
+      args.setValue(value);
+      sendBase("echoRuntimeFail", args);
+    }
+
+    public java.lang.String recv_echoRuntimeFail() throws org.apache.thrift.TException
+    {
+      echoRuntimeFail_result result = new echoRuntimeFail_result();
+      receiveBase(result, "echoRuntimeFail");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "echoRuntimeFail failed: unknown result");
+    }
+
+    public void onewayRuntimeFail(java.lang.String value) throws org.apache.thrift.TException
+    {
+      send_onewayRuntimeFail(value);
+    }
+
+    public void send_onewayRuntimeFail(java.lang.String value) throws org.apache.thrift.TException
+    {
+      onewayRuntimeFail_args args = new onewayRuntimeFail_args();
+      args.setValue(value);
+      sendBaseOneway("onewayRuntimeFail", args);
+    }
+
+  }
+  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 echoPass(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      echoPass_call method_call = new echoPass_call(value, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class echoPass_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
+      private java.lang.String value;
+      public echoPass_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 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.value = value;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("echoPass", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        echoPass_args args = new echoPass_args();
+        args.setValue(value);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.lang.String getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.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_echoPass();
+      }
+    }
+
+    public void onewayPass(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      onewayPass_call method_call = new onewayPass_call(value, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class onewayPass_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private java.lang.String value;
+      public onewayPass_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
+        this.value = value;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("onewayPass", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        onewayPass_args args = new onewayPass_args();
+        args.setValue(value);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.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 null;
+      }
+    }
+
+    public void echoFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      echoFail_call method_call = new echoFail_call(value, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class echoFail_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
+      private java.lang.String value;
+      public echoFail_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 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.value = value;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("echoFail", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        echoFail_args args = new echoFail_args();
+        args.setValue(value);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.lang.String getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.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_echoFail();
+      }
+    }
+
+    public void onewayFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      onewayFail_call method_call = new onewayFail_call(value, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class onewayFail_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private java.lang.String value;
+      public onewayFail_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
+        this.value = value;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("onewayFail", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        onewayFail_args args = new onewayFail_args();
+        args.setValue(value);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.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 null;
+      }
+    }
+
+    public void echoRuntimeFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      echoRuntimeFail_call method_call = new echoRuntimeFail_call(value, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class echoRuntimeFail_call extends org.apache.thrift.async.TAsyncMethodCall<java.lang.String> {
+      private java.lang.String value;
+      public echoRuntimeFail_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> 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.value = value;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("echoRuntimeFail", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        echoRuntimeFail_args args = new echoRuntimeFail_args();
+        args.setValue(value);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public java.lang.String getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.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_echoRuntimeFail();
+      }
+    }
+
+    public void onewayRuntimeFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      onewayRuntimeFail_call method_call = new onewayRuntimeFail_call(value, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class onewayRuntimeFail_call extends org.apache.thrift.async.TAsyncMethodCall<Void> {
+      private java.lang.String value;
+      public onewayRuntimeFail_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback<Void> 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, true);
+        this.value = value;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("onewayRuntimeFail", org.apache.thrift.protocol.TMessageType.ONEWAY, 0));
+        onewayRuntimeFail_args args = new onewayRuntimeFail_args();
+        args.setValue(value);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public Void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new java.lang.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 null;
+      }
+    }
+
+  }
+
+  public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>>()));
+    }
+
+    protected Processor(I iface, java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends Iface> java.util.Map<java.lang.String,  org.apache.thrift.ProcessFunction<I, ? extends org.apache.thrift.TBase>> getProcessMap(java.util.Map<java.lang.String, org.apache.thrift.ProcessFunction<I, ? extends  org.apache.thrift.TBase>> processMap) {
+      processMap.put("echoPass", new echoPass());
+      processMap.put("onewayPass", new onewayPass());
+      processMap.put("echoFail", new echoFail());
+      processMap.put("onewayFail", new onewayFail());
+      processMap.put("echoRuntimeFail", new echoRuntimeFail());
+      processMap.put("onewayRuntimeFail", new onewayRuntimeFail());
+      return processMap;
+    }
+
+    public static class echoPass<I extends Iface> extends org.apache.thrift.ProcessFunction<I, echoPass_args> {
+      public echoPass() {
+        super("echoPass");
+      }
+
+      public echoPass_args getEmptyArgsInstance() {
+        return new echoPass_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public echoPass_result getResult(I iface, echoPass_args args) throws org.apache.thrift.TException {
+        echoPass_result result = new echoPass_result();
+        result.success = iface.echoPass(args.value);
+        return result;
+      }
+    }
+
+    public static class onewayPass<I extends Iface> extends org.apache.thrift.ProcessFunction<I, onewayPass_args> {
+      public onewayPass() {
+        super("onewayPass");
+      }
+
+      public onewayPass_args getEmptyArgsInstance() {
+        return new onewayPass_args();
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public org.apache.thrift.TBase getResult(I iface, onewayPass_args args) throws org.apache.thrift.TException {
+        iface.onewayPass(args.value);
+        return null;
+      }
+    }
+
+    public static class echoFail<I extends Iface> extends org.apache.thrift.ProcessFunction<I, echoFail_args> {
+      public echoFail() {
+        super("echoFail");
+      }
+
+      public echoFail_args getEmptyArgsInstance() {
+        return new echoFail_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public echoFail_result getResult(I iface, echoFail_args args) throws org.apache.thrift.TException {
+        echoFail_result result = new echoFail_result();
+        result.success = iface.echoFail(args.value);
+        return result;
+      }
+    }
+
+    public static class onewayFail<I extends Iface> extends org.apache.thrift.ProcessFunction<I, onewayFail_args> {
+      public onewayFail() {
+        super("onewayFail");
+      }
+
+      public onewayFail_args getEmptyArgsInstance() {
+        return new onewayFail_args();
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public org.apache.thrift.TBase getResult(I iface, onewayFail_args args) throws org.apache.thrift.TException {
+        iface.onewayFail(args.value);
+        return null;
+      }
+    }
+
+    public static class echoRuntimeFail<I extends Iface> extends org.apache.thrift.ProcessFunction<I, echoRuntimeFail_args> {
+      public echoRuntimeFail() {
+        super("echoRuntimeFail");
+      }
+
+      public echoRuntimeFail_args getEmptyArgsInstance() {
+        return new echoRuntimeFail_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public echoRuntimeFail_result getResult(I iface, echoRuntimeFail_args args) throws org.apache.thrift.TException {
+        echoRuntimeFail_result result = new echoRuntimeFail_result();
+        result.success = iface.echoRuntimeFail(args.value);
+        return result;
+      }
+    }
+
+    public static class onewayRuntimeFail<I extends Iface> extends org.apache.thrift.ProcessFunction<I, onewayRuntimeFail_args> {
+      public onewayRuntimeFail() {
+        super("onewayRuntimeFail");
+      }
+
+      public onewayRuntimeFail_args getEmptyArgsInstance() {
+        return new onewayRuntimeFail_args();
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public org.apache.thrift.TBase getResult(I iface, onewayRuntimeFail_args args) throws org.apache.thrift.TException {
+        iface.onewayRuntimeFail(args.value);
+        return null;
+      }
+    }
+
+  }
+
+  public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
+    private static final org.slf4j.Logger _LOGGER = org.slf4j.LoggerFactory.getLogger(AsyncProcessor.class.getName());
+    public AsyncProcessor(I iface) {
+      super(iface, getProcessMap(new java.util.HashMap<java.lang.String, org.apache.thrift.AsyncProcessFunction<I, ? extends org.apache.thrift.TBase, ?>>()));
+    }
+
+    protected AsyncProcessor(I iface, java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      super(iface, getProcessMap(processMap));
+    }
+
+    private static <I extends AsyncIface> java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase,?>> getProcessMap(java.util.Map<java.lang.String,  org.apache.thrift.AsyncProcessFunction<I, ? extends  org.apache.thrift.TBase, ?>> processMap) {
+      processMap.put("echoPass", new echoPass());
+      processMap.put("onewayPass", new onewayPass());
+      processMap.put("echoFail", new echoFail());
+      processMap.put("onewayFail", new onewayFail());
+      processMap.put("echoRuntimeFail", new echoRuntimeFail());
+      processMap.put("onewayRuntimeFail", new onewayRuntimeFail());
+      return processMap;
+    }
+
+    public static class echoPass<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, echoPass_args, java.lang.String> {
+      public echoPass() {
+        super("echoPass");
+      }
+
+      public echoPass_args getEmptyArgsInstance() {
+        return new echoPass_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
+          public void onComplete(java.lang.String o) {
+            echoPass_result result = new echoPass_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            echoPass_result result = new echoPass_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, echoPass_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+        iface.echoPass(args.value,resultHandler);
+      }
+    }
+
+    public static class onewayPass<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, onewayPass_args, Void> {
+      public onewayPass() {
+        super("onewayPass");
+      }
+
+      public onewayPass_args getEmptyArgsInstance() {
+        return new onewayPass_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, onewayPass_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.onewayPass(args.value,resultHandler);
+      }
+    }
+
+    public static class echoFail<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, echoFail_args, java.lang.String> {
+      public echoFail() {
+        super("echoFail");
+      }
+
+      public echoFail_args getEmptyArgsInstance() {
+        return new echoFail_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
+          public void onComplete(java.lang.String o) {
+            echoFail_result result = new echoFail_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            echoFail_result result = new echoFail_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, echoFail_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+        iface.echoFail(args.value,resultHandler);
+      }
+    }
+
+    public static class onewayFail<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, onewayFail_args, Void> {
+      public onewayFail() {
+        super("onewayFail");
+      }
+
+      public onewayFail_args getEmptyArgsInstance() {
+        return new onewayFail_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, onewayFail_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.onewayFail(args.value,resultHandler);
+      }
+    }
+
+    public static class echoRuntimeFail<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, echoRuntimeFail_args, java.lang.String> {
+      public echoRuntimeFail() {
+        super("echoRuntimeFail");
+      }
+
+      public echoRuntimeFail_args getEmptyArgsInstance() {
+        return new echoRuntimeFail_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<java.lang.String> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<java.lang.String>() { 
+          public void onComplete(java.lang.String o) {
+            echoRuntimeFail_result result = new echoRuntimeFail_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb, result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+            } catch (org.apache.thrift.transport.TTransportException e) {
+              _LOGGER.error("TTransportException writing to internal frame buffer", e);
+              fb.close();
+            } catch (java.lang.Exception e) {
+              _LOGGER.error("Exception writing to internal frame buffer", e);
+              onError(e);
+            }
+          }
+          public void onError(java.lang.Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TSerializable msg;
+            echoRuntimeFail_result result = new echoRuntimeFail_result();
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+              return;
+            } else if (e instanceof org.apache.thrift.TApplicationException) {
+              _LOGGER.error("TApplicationException inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TApplicationException)e;
+            } else {
+              _LOGGER.error("Exception inside handler", e);
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+            } catch (java.lang.Exception ex) {
+              _LOGGER.error("Exception writing to internal frame buffer", ex);
+              fb.close();
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, echoRuntimeFail_args args, org.apache.thrift.async.AsyncMethodCallback<java.lang.String> resultHandler) throws org.apache.thrift.TException {
+        iface.echoRuntimeFail(args.value,resultHandler);
+      }
+    }
+
+    public static class onewayRuntimeFail<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, onewayRuntimeFail_args, Void> {
+      public onewayRuntimeFail() {
+        super("onewayRuntimeFail");
+      }
+
+      public onewayRuntimeFail_args getEmptyArgsInstance() {
+        return new onewayRuntimeFail_args();
+      }
+
+      public org.apache.thrift.async.AsyncMethodCallback<Void> getResultHandler(final org.apache.thrift.server.AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new org.apache.thrift.async.AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(java.lang.Exception e) {
+            if (e instanceof org.apache.thrift.transport.TTransportException) {
+              _LOGGER.error("TTransportException inside handler", e);
+              fb.close();
+            } else {
+              _LOGGER.error("Exception inside oneway handler", e);
+            }
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, onewayRuntimeFail_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws org.apache.thrift.TException {
+        iface.onewayRuntimeFail(args.value,resultHandler);
+      }
+    }
+
+  }
+
+  public static class echoPass_args implements org.apache.thrift.TBase<echoPass_args, echoPass_args._Fields>, java.io.Serializable, Cloneable, Comparable<echoPass_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("echoPass_args");
+
+    private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new echoPass_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new echoPass_argsTupleSchemeFactory();
+
+    public java.lang.String value; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      VALUE((short)1, "value");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.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: // VALUE
+            return VALUE;
+          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 java.lang.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(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(echoPass_args.class, metaDataMap);
+    }
+
+    public echoPass_args() {
+    }
+
+    public echoPass_args(
+      java.lang.String value)
+    {
+      this();
+      this.value = value;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public echoPass_args(echoPass_args other) {
+      if (other.isSetValue()) {
+        this.value = other.value;
+      }
+    }
+
+    public echoPass_args deepCopy() {
+      return new echoPass_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.value = null;
+    }
+
+    public java.lang.String getValue() {
+      return this.value;
+    }
+
+    public echoPass_args setValue(java.lang.String value) {
+      this.value = value;
+      return this;
+    }
+
+    public void unsetValue() {
+      this.value = null;
+    }
+
+    /** Returns true if field value is set (has been assigned a value) and false otherwise */
+    public boolean isSetValue() {
+      return this.value != null;
+    }
+
+    public void setValueIsSet(boolean value) {
+      if (!value) {
+        this.value = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case VALUE:
+        if (value == null) {
+          unsetValue();
+        } else {
+          setValue((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case VALUE:
+        return getValue();
+
+      }
+      throw new java.lang.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 java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case VALUE:
+        return isSetValue();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof echoPass_args)
+        return this.equals((echoPass_args)that);
+      return false;
+    }
+
+    public boolean equals(echoPass_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_value = true && this.isSetValue();
+      boolean that_present_value = true && that.isSetValue();
+      if (this_present_value || that_present_value) {
+        if (!(this_present_value && that_present_value))
+          return false;
+        if (!this.value.equals(that.value))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetValue()) ? 131071 : 524287);
+      if (isSetValue())
+        hashCode = hashCode * 8191 + value.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(echoPass_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetValue()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+        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 {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("echoPass_args(");
+      boolean first = true;
+
+      sb.append("value:");
+      if (this.value == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.value);
+      }
+      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, java.lang.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 echoPass_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoPass_argsStandardScheme getScheme() {
+        return new echoPass_argsStandardScheme();
+      }
+    }
+
+    private static class echoPass_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<echoPass_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, echoPass_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: // VALUE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.value = iprot.readString();
+                struct.setValueIsSet(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, echoPass_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.value != null) {
+          oprot.writeFieldBegin(VALUE_FIELD_DESC);
+          oprot.writeString(struct.value);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class echoPass_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoPass_argsTupleScheme getScheme() {
+        return new echoPass_argsTupleScheme();
+      }
+    }
+
+    private static class echoPass_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<echoPass_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, echoPass_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetValue()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetValue()) {
+          oprot.writeString(struct.value);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, echoPass_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.value = iprot.readString();
+          struct.setValueIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class echoPass_result implements org.apache.thrift.TBase<echoPass_result, echoPass_result._Fields>, java.io.Serializable, Cloneable, Comparable<echoPass_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("echoPass_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new echoPass_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new echoPass_resultTupleSchemeFactory();
+
+    public java.lang.String success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.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;
+          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 java.lang.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(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(echoPass_result.class, metaDataMap);
+    }
+
+    public echoPass_result() {
+    }
+
+    public echoPass_result(
+      java.lang.String success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public echoPass_result(echoPass_result other) {
+      if (other.isSetSuccess()) {
+        this.success = other.success;
+      }
+    }
+
+    public echoPass_result deepCopy() {
+      return new echoPass_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public java.lang.String getSuccess() {
+      return this.success;
+    }
+
+    public echoPass_result setSuccess(java.lang.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 void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.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 java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof echoPass_result)
+        return this.equals((echoPass_result)that);
+      return false;
+    }
+
+    public boolean equals(echoPass_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      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;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(echoPass_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        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 {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("echoPass_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      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, java.lang.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 echoPass_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoPass_resultStandardScheme getScheme() {
+        return new echoPass_resultStandardScheme();
+      }
+    }
+
+    private static class echoPass_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<echoPass_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, echoPass_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;
+            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, echoPass_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();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class echoPass_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoPass_resultTupleScheme getScheme() {
+        return new echoPass_resultTupleScheme();
+      }
+    }
+
+    private static class echoPass_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<echoPass_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, echoPass_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          oprot.writeString(struct.success);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, echoPass_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = iprot.readString();
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class onewayPass_args implements org.apache.thrift.TBase<onewayPass_args, onewayPass_args._Fields>, java.io.Serializable, Cloneable, Comparable<onewayPass_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("onewayPass_args");
+
+    private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new onewayPass_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new onewayPass_argsTupleSchemeFactory();
+
+    public java.lang.String value; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      VALUE((short)1, "value");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.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: // VALUE
+            return VALUE;
+          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 java.lang.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(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(onewayPass_args.class, metaDataMap);
+    }
+
+    public onewayPass_args() {
+    }
+
+    public onewayPass_args(
+      java.lang.String value)
+    {
+      this();
+      this.value = value;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public onewayPass_args(onewayPass_args other) {
+      if (other.isSetValue()) {
+        this.value = other.value;
+      }
+    }
+
+    public onewayPass_args deepCopy() {
+      return new onewayPass_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.value = null;
+    }
+
+    public java.lang.String getValue() {
+      return this.value;
+    }
+
+    public onewayPass_args setValue(java.lang.String value) {
+      this.value = value;
+      return this;
+    }
+
+    public void unsetValue() {
+      this.value = null;
+    }
+
+    /** Returns true if field value is set (has been assigned a value) and false otherwise */
+    public boolean isSetValue() {
+      return this.value != null;
+    }
+
+    public void setValueIsSet(boolean value) {
+      if (!value) {
+        this.value = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case VALUE:
+        if (value == null) {
+          unsetValue();
+        } else {
+          setValue((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case VALUE:
+        return getValue();
+
+      }
+      throw new java.lang.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 java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case VALUE:
+        return isSetValue();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof onewayPass_args)
+        return this.equals((onewayPass_args)that);
+      return false;
+    }
+
+    public boolean equals(onewayPass_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_value = true && this.isSetValue();
+      boolean that_present_value = true && that.isSetValue();
+      if (this_present_value || that_present_value) {
+        if (!(this_present_value && that_present_value))
+          return false;
+        if (!this.value.equals(that.value))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetValue()) ? 131071 : 524287);
+      if (isSetValue())
+        hashCode = hashCode * 8191 + value.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(onewayPass_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetValue()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+        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 {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("onewayPass_args(");
+      boolean first = true;
+
+      sb.append("value:");
+      if (this.value == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.value);
+      }
+      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, java.lang.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 onewayPass_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public onewayPass_argsStandardScheme getScheme() {
+        return new onewayPass_argsStandardScheme();
+      }
+    }
+
+    private static class onewayPass_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<onewayPass_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, onewayPass_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: // VALUE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.value = iprot.readString();
+                struct.setValueIsSet(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, onewayPass_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.value != null) {
+          oprot.writeFieldBegin(VALUE_FIELD_DESC);
+          oprot.writeString(struct.value);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class onewayPass_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public onewayPass_argsTupleScheme getScheme() {
+        return new onewayPass_argsTupleScheme();
+      }
+    }
+
+    private static class onewayPass_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<onewayPass_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, onewayPass_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetValue()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetValue()) {
+          oprot.writeString(struct.value);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, onewayPass_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.value = iprot.readString();
+          struct.setValueIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class echoFail_args implements org.apache.thrift.TBase<echoFail_args, echoFail_args._Fields>, java.io.Serializable, Cloneable, Comparable<echoFail_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("echoFail_args");
+
+    private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new echoFail_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new echoFail_argsTupleSchemeFactory();
+
+    public java.lang.String value; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      VALUE((short)1, "value");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.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: // VALUE
+            return VALUE;
+          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 java.lang.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(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(echoFail_args.class, metaDataMap);
+    }
+
+    public echoFail_args() {
+    }
+
+    public echoFail_args(
+      java.lang.String value)
+    {
+      this();
+      this.value = value;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public echoFail_args(echoFail_args other) {
+      if (other.isSetValue()) {
+        this.value = other.value;
+      }
+    }
+
+    public echoFail_args deepCopy() {
+      return new echoFail_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.value = null;
+    }
+
+    public java.lang.String getValue() {
+      return this.value;
+    }
+
+    public echoFail_args setValue(java.lang.String value) {
+      this.value = value;
+      return this;
+    }
+
+    public void unsetValue() {
+      this.value = null;
+    }
+
+    /** Returns true if field value is set (has been assigned a value) and false otherwise */
+    public boolean isSetValue() {
+      return this.value != null;
+    }
+
+    public void setValueIsSet(boolean value) {
+      if (!value) {
+        this.value = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case VALUE:
+        if (value == null) {
+          unsetValue();
+        } else {
+          setValue((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case VALUE:
+        return getValue();
+
+      }
+      throw new java.lang.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 java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case VALUE:
+        return isSetValue();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof echoFail_args)
+        return this.equals((echoFail_args)that);
+      return false;
+    }
+
+    public boolean equals(echoFail_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_value = true && this.isSetValue();
+      boolean that_present_value = true && that.isSetValue();
+      if (this_present_value || that_present_value) {
+        if (!(this_present_value && that_present_value))
+          return false;
+        if (!this.value.equals(that.value))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetValue()) ? 131071 : 524287);
+      if (isSetValue())
+        hashCode = hashCode * 8191 + value.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(echoFail_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetValue()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+        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 {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("echoFail_args(");
+      boolean first = true;
+
+      sb.append("value:");
+      if (this.value == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.value);
+      }
+      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, java.lang.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 echoFail_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoFail_argsStandardScheme getScheme() {
+        return new echoFail_argsStandardScheme();
+      }
+    }
+
+    private static class echoFail_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<echoFail_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, echoFail_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: // VALUE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.value = iprot.readString();
+                struct.setValueIsSet(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, echoFail_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.value != null) {
+          oprot.writeFieldBegin(VALUE_FIELD_DESC);
+          oprot.writeString(struct.value);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class echoFail_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoFail_argsTupleScheme getScheme() {
+        return new echoFail_argsTupleScheme();
+      }
+    }
+
+    private static class echoFail_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<echoFail_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, echoFail_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetValue()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetValue()) {
+          oprot.writeString(struct.value);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, echoFail_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.value = iprot.readString();
+          struct.setValueIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class echoFail_result implements org.apache.thrift.TBase<echoFail_result, echoFail_result._Fields>, java.io.Serializable, Cloneable, Comparable<echoFail_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("echoFail_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new echoFail_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new echoFail_resultTupleSchemeFactory();
+
+    public java.lang.String success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.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;
+          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 java.lang.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(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(echoFail_result.class, metaDataMap);
+    }
+
+    public echoFail_result() {
+    }
+
+    public echoFail_result(
+      java.lang.String success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public echoFail_result(echoFail_result other) {
+      if (other.isSetSuccess()) {
+        this.success = other.success;
+      }
+    }
+
+    public echoFail_result deepCopy() {
+      return new echoFail_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public java.lang.String getSuccess() {
+      return this.success;
+    }
+
+    public echoFail_result setSuccess(java.lang.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 void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.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 java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof echoFail_result)
+        return this.equals((echoFail_result)that);
+      return false;
+    }
+
+    public boolean equals(echoFail_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      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;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(echoFail_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        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 {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("echoFail_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      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, java.lang.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 echoFail_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoFail_resultStandardScheme getScheme() {
+        return new echoFail_resultStandardScheme();
+      }
+    }
+
+    private static class echoFail_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<echoFail_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, echoFail_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;
+            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, echoFail_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();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class echoFail_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoFail_resultTupleScheme getScheme() {
+        return new echoFail_resultTupleScheme();
+      }
+    }
+
+    private static class echoFail_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<echoFail_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, echoFail_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          oprot.writeString(struct.success);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, echoFail_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = iprot.readString();
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class onewayFail_args implements org.apache.thrift.TBase<onewayFail_args, onewayFail_args._Fields>, java.io.Serializable, Cloneable, Comparable<onewayFail_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("onewayFail_args");
+
+    private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new onewayFail_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new onewayFail_argsTupleSchemeFactory();
+
+    public java.lang.String value; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      VALUE((short)1, "value");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.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: // VALUE
+            return VALUE;
+          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 java.lang.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(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(onewayFail_args.class, metaDataMap);
+    }
+
+    public onewayFail_args() {
+    }
+
+    public onewayFail_args(
+      java.lang.String value)
+    {
+      this();
+      this.value = value;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public onewayFail_args(onewayFail_args other) {
+      if (other.isSetValue()) {
+        this.value = other.value;
+      }
+    }
+
+    public onewayFail_args deepCopy() {
+      return new onewayFail_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.value = null;
+    }
+
+    public java.lang.String getValue() {
+      return this.value;
+    }
+
+    public onewayFail_args setValue(java.lang.String value) {
+      this.value = value;
+      return this;
+    }
+
+    public void unsetValue() {
+      this.value = null;
+    }
+
+    /** Returns true if field value is set (has been assigned a value) and false otherwise */
+    public boolean isSetValue() {
+      return this.value != null;
+    }
+
+    public void setValueIsSet(boolean value) {
+      if (!value) {
+        this.value = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case VALUE:
+        if (value == null) {
+          unsetValue();
+        } else {
+          setValue((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case VALUE:
+        return getValue();
+
+      }
+      throw new java.lang.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 java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case VALUE:
+        return isSetValue();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof onewayFail_args)
+        return this.equals((onewayFail_args)that);
+      return false;
+    }
+
+    public boolean equals(onewayFail_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_value = true && this.isSetValue();
+      boolean that_present_value = true && that.isSetValue();
+      if (this_present_value || that_present_value) {
+        if (!(this_present_value && that_present_value))
+          return false;
+        if (!this.value.equals(that.value))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetValue()) ? 131071 : 524287);
+      if (isSetValue())
+        hashCode = hashCode * 8191 + value.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(onewayFail_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetValue()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+        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 {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("onewayFail_args(");
+      boolean first = true;
+
+      sb.append("value:");
+      if (this.value == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.value);
+      }
+      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, java.lang.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 onewayFail_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public onewayFail_argsStandardScheme getScheme() {
+        return new onewayFail_argsStandardScheme();
+      }
+    }
+
+    private static class onewayFail_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<onewayFail_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, onewayFail_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: // VALUE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.value = iprot.readString();
+                struct.setValueIsSet(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, onewayFail_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.value != null) {
+          oprot.writeFieldBegin(VALUE_FIELD_DESC);
+          oprot.writeString(struct.value);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class onewayFail_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public onewayFail_argsTupleScheme getScheme() {
+        return new onewayFail_argsTupleScheme();
+      }
+    }
+
+    private static class onewayFail_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<onewayFail_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, onewayFail_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetValue()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetValue()) {
+          oprot.writeString(struct.value);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, onewayFail_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.value = iprot.readString();
+          struct.setValueIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class echoRuntimeFail_args implements org.apache.thrift.TBase<echoRuntimeFail_args, echoRuntimeFail_args._Fields>, java.io.Serializable, Cloneable, Comparable<echoRuntimeFail_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("echoRuntimeFail_args");
+
+    private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new echoRuntimeFail_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new echoRuntimeFail_argsTupleSchemeFactory();
+
+    public java.lang.String value; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      VALUE((short)1, "value");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.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: // VALUE
+            return VALUE;
+          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 java.lang.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(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(echoRuntimeFail_args.class, metaDataMap);
+    }
+
+    public echoRuntimeFail_args() {
+    }
+
+    public echoRuntimeFail_args(
+      java.lang.String value)
+    {
+      this();
+      this.value = value;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public echoRuntimeFail_args(echoRuntimeFail_args other) {
+      if (other.isSetValue()) {
+        this.value = other.value;
+      }
+    }
+
+    public echoRuntimeFail_args deepCopy() {
+      return new echoRuntimeFail_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.value = null;
+    }
+
+    public java.lang.String getValue() {
+      return this.value;
+    }
+
+    public echoRuntimeFail_args setValue(java.lang.String value) {
+      this.value = value;
+      return this;
+    }
+
+    public void unsetValue() {
+      this.value = null;
+    }
+
+    /** Returns true if field value is set (has been assigned a value) and false otherwise */
+    public boolean isSetValue() {
+      return this.value != null;
+    }
+
+    public void setValueIsSet(boolean value) {
+      if (!value) {
+        this.value = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case VALUE:
+        if (value == null) {
+          unsetValue();
+        } else {
+          setValue((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case VALUE:
+        return getValue();
+
+      }
+      throw new java.lang.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 java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case VALUE:
+        return isSetValue();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof echoRuntimeFail_args)
+        return this.equals((echoRuntimeFail_args)that);
+      return false;
+    }
+
+    public boolean equals(echoRuntimeFail_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_value = true && this.isSetValue();
+      boolean that_present_value = true && that.isSetValue();
+      if (this_present_value || that_present_value) {
+        if (!(this_present_value && that_present_value))
+          return false;
+        if (!this.value.equals(that.value))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetValue()) ? 131071 : 524287);
+      if (isSetValue())
+        hashCode = hashCode * 8191 + value.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(echoRuntimeFail_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetValue()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+        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 {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("echoRuntimeFail_args(");
+      boolean first = true;
+
+      sb.append("value:");
+      if (this.value == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.value);
+      }
+      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, java.lang.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 echoRuntimeFail_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoRuntimeFail_argsStandardScheme getScheme() {
+        return new echoRuntimeFail_argsStandardScheme();
+      }
+    }
+
+    private static class echoRuntimeFail_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<echoRuntimeFail_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, echoRuntimeFail_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: // VALUE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.value = iprot.readString();
+                struct.setValueIsSet(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, echoRuntimeFail_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.value != null) {
+          oprot.writeFieldBegin(VALUE_FIELD_DESC);
+          oprot.writeString(struct.value);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class echoRuntimeFail_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoRuntimeFail_argsTupleScheme getScheme() {
+        return new echoRuntimeFail_argsTupleScheme();
+      }
+    }
+
+    private static class echoRuntimeFail_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<echoRuntimeFail_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, echoRuntimeFail_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetValue()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetValue()) {
+          oprot.writeString(struct.value);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, echoRuntimeFail_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.value = iprot.readString();
+          struct.setValueIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class echoRuntimeFail_result implements org.apache.thrift.TBase<echoRuntimeFail_result, echoRuntimeFail_result._Fields>, java.io.Serializable, Cloneable, Comparable<echoRuntimeFail_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("echoRuntimeFail_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.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new echoRuntimeFail_resultStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new echoRuntimeFail_resultTupleSchemeFactory();
+
+    public java.lang.String success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.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;
+          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 java.lang.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(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.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)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(echoRuntimeFail_result.class, metaDataMap);
+    }
+
+    public echoRuntimeFail_result() {
+    }
+
+    public echoRuntimeFail_result(
+      java.lang.String success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public echoRuntimeFail_result(echoRuntimeFail_result other) {
+      if (other.isSetSuccess()) {
+        this.success = other.success;
+      }
+    }
+
+    public echoRuntimeFail_result deepCopy() {
+      return new echoRuntimeFail_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public java.lang.String getSuccess() {
+      return this.success;
+    }
+
+    public echoRuntimeFail_result setSuccess(java.lang.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 void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      throw new java.lang.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 java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof echoRuntimeFail_result)
+        return this.equals((echoRuntimeFail_result)that);
+      return false;
+    }
+
+    public boolean equals(echoRuntimeFail_result that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      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;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetSuccess()) ? 131071 : 524287);
+      if (isSetSuccess())
+        hashCode = hashCode * 8191 + success.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(echoRuntimeFail_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        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 {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+      }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("echoRuntimeFail_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      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, java.lang.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 echoRuntimeFail_resultStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoRuntimeFail_resultStandardScheme getScheme() {
+        return new echoRuntimeFail_resultStandardScheme();
+      }
+    }
+
+    private static class echoRuntimeFail_resultStandardScheme extends org.apache.thrift.scheme.StandardScheme<echoRuntimeFail_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, echoRuntimeFail_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;
+            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, echoRuntimeFail_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();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class echoRuntimeFail_resultTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public echoRuntimeFail_resultTupleScheme getScheme() {
+        return new echoRuntimeFail_resultTupleScheme();
+      }
+    }
+
+    private static class echoRuntimeFail_resultTupleScheme extends org.apache.thrift.scheme.TupleScheme<echoRuntimeFail_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, echoRuntimeFail_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          oprot.writeString(struct.success);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, echoRuntimeFail_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = iprot.readString();
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  public static class onewayRuntimeFail_args implements org.apache.thrift.TBase<onewayRuntimeFail_args, onewayRuntimeFail_args._Fields>, java.io.Serializable, Cloneable, Comparable<onewayRuntimeFail_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("onewayRuntimeFail_args");
+
+    private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final org.apache.thrift.scheme.SchemeFactory STANDARD_SCHEME_FACTORY = new onewayRuntimeFail_argsStandardSchemeFactory();
+    private static final org.apache.thrift.scheme.SchemeFactory TUPLE_SCHEME_FACTORY = new onewayRuntimeFail_argsTupleSchemeFactory();
+
+    public java.lang.String value; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      VALUE((short)1, "value");
+
+      private static final java.util.Map<java.lang.String, _Fields> byName = new java.util.HashMap<java.lang.String, _Fields>();
+
+      static {
+        for (_Fields field : java.util.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: // VALUE
+            return VALUE;
+          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 java.lang.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(java.lang.String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final java.lang.String _fieldName;
+
+      _Fields(short thriftId, java.lang.String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public java.lang.String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(onewayRuntimeFail_args.class, metaDataMap);
+    }
+
+    public onewayRuntimeFail_args() {
+    }
+
+    public onewayRuntimeFail_args(
+      java.lang.String value)
+    {
+      this();
+      this.value = value;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public onewayRuntimeFail_args(onewayRuntimeFail_args other) {
+      if (other.isSetValue()) {
+        this.value = other.value;
+      }
+    }
+
+    public onewayRuntimeFail_args deepCopy() {
+      return new onewayRuntimeFail_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.value = null;
+    }
+
+    public java.lang.String getValue() {
+      return this.value;
+    }
+
+    public onewayRuntimeFail_args setValue(java.lang.String value) {
+      this.value = value;
+      return this;
+    }
+
+    public void unsetValue() {
+      this.value = null;
+    }
+
+    /** Returns true if field value is set (has been assigned a value) and false otherwise */
+    public boolean isSetValue() {
+      return this.value != null;
+    }
+
+    public void setValueIsSet(boolean value) {
+      if (!value) {
+        this.value = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, java.lang.Object value) {
+      switch (field) {
+      case VALUE:
+        if (value == null) {
+          unsetValue();
+        } else {
+          setValue((java.lang.String)value);
+        }
+        break;
+
+      }
+    }
+
+    public java.lang.Object getFieldValue(_Fields field) {
+      switch (field) {
+      case VALUE:
+        return getValue();
+
+      }
+      throw new java.lang.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 java.lang.IllegalArgumentException();
+      }
+
+      switch (field) {
+      case VALUE:
+        return isSetValue();
+      }
+      throw new java.lang.IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(java.lang.Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof onewayRuntimeFail_args)
+        return this.equals((onewayRuntimeFail_args)that);
+      return false;
+    }
+
+    public boolean equals(onewayRuntimeFail_args that) {
+      if (that == null)
+        return false;
+      if (this == that)
+        return true;
+
+      boolean this_present_value = true && this.isSetValue();
+      boolean that_present_value = true && that.isSetValue();
+      if (this_present_value || that_present_value) {
+        if (!(this_present_value && that_present_value))
+          return false;
+        if (!this.value.equals(that.value))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int hashCode = 1;
+
+      hashCode = hashCode * 8191 + ((isSetValue()) ? 131071 : 524287);
+      if (isSetValue())
+        hashCode = hashCode * 8191 + value.hashCode();
+
+      return hashCode;
+    }
+
+    @Override
+    public int compareTo(onewayRuntimeFail_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = java.lang.Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetValue()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+        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 {
+      scheme(iprot).read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      scheme(oprot).write(oprot, this);
+    }
+
+    @Override
+    public java.lang.String toString() {
+      java.lang.StringBuilder sb = new java.lang.StringBuilder("onewayRuntimeFail_args(");
+      boolean first = true;
+
+      sb.append("value:");
+      if (this.value == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.value);
+      }
+      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, java.lang.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 onewayRuntimeFail_argsStandardSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public onewayRuntimeFail_argsStandardScheme getScheme() {
+        return new onewayRuntimeFail_argsStandardScheme();
+      }
+    }
+
+    private static class onewayRuntimeFail_argsStandardScheme extends org.apache.thrift.scheme.StandardScheme<onewayRuntimeFail_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, onewayRuntimeFail_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: // VALUE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.value = iprot.readString();
+                struct.setValueIsSet(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, onewayRuntimeFail_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.value != null) {
+          oprot.writeFieldBegin(VALUE_FIELD_DESC);
+          oprot.writeString(struct.value);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class onewayRuntimeFail_argsTupleSchemeFactory implements org.apache.thrift.scheme.SchemeFactory {
+      public onewayRuntimeFail_argsTupleScheme getScheme() {
+        return new onewayRuntimeFail_argsTupleScheme();
+      }
+    }
+
+    private static class onewayRuntimeFail_argsTupleScheme extends org.apache.thrift.scheme.TupleScheme<onewayRuntimeFail_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, onewayRuntimeFail_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol oprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet optionals = new java.util.BitSet();
+        if (struct.isSetValue()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetValue()) {
+          oprot.writeString(struct.value);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, onewayRuntimeFail_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
+        java.util.BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.value = iprot.readString();
+          struct.setValueIsSet(true);
+        }
+      }
+    }
+
+    private static <S extends org.apache.thrift.scheme.IScheme> S scheme(org.apache.thrift.protocol.TProtocol proto) {
+      return (org.apache.thrift.scheme.StandardScheme.class.equals(proto.getScheme()) ? STANDARD_SCHEME_FACTORY : TUPLE_SCHEME_FACTORY).getScheme();
+    }
+  }
+
+  private static void unusedMethod() {}
+}
diff --git a/test/src/main/scripts/generate-thrift.sh b/test/src/main/scripts/generate-thrift.sh
new file mode 100755
index 0000000..061ad4f
--- /dev/null
+++ b/test/src/main/scripts/generate-thrift.sh
@@ -0,0 +1,24 @@
+#! /usr/bin/env bash
+
+# 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 script will regenerate the thrift code for accumulo-trace.
+INCLUDED_MODULES=(-)
+BASE_OUTPUT_PACKAGE='org.apache.accumulo'
+PACKAGES_TO_GENERATE=(test.rpc)
+
+. ../core/src/main/scripts/generate-thrift.sh
+
diff --git a/test/src/main/thrift/test.thrift b/test/src/main/thrift/test.thrift
new file mode 100644
index 0000000..7e4d955
--- /dev/null
+++ b/test/src/main/thrift/test.thrift
@@ -0,0 +1,30 @@
+/*
+* 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.
+*/
+
+namespace java org.apache.accumulo.test.rpc.thrift
+
+service SimpleThriftService
+{
+  string echoPass(1:string value)
+  oneway void onewayPass(1:string value)
+
+  string echoFail(1:string value)
+  oneway void onewayFail(1:string value)
+
+  string echoRuntimeFail(1:string value)
+  oneway void onewayRuntimeFail(1:string value)
+}