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 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 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 tableMap 2:list tServerInfo - 3:map badTServers + 3:map 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 @@ false - 0.9.3 + 0.10.0 3.4.6 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(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 service(final I instance, final TBaseProcessor processor) { - final Map> processorView = processor.getProcessMapView(); - final Set onewayMethods = getOnewayMethods(processorView); - log.debug("Found oneway Thrift methods: " + onewayMethods); - - InvocationHandler handler = getInvocationHandler(instance, onewayMethods); + public static 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 RpcServerInvocationHandler getInvocationHandler(final T instance, - final Set onewayMethods) { + protected static RpcServerInvocationHandler getInvocationHandler(final T instance) { return new RpcServerInvocationHandler(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 getOnewayMethods(Map 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 onewayMethods = new HashSet<>(); - for (Entry 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 createProcessFunction(String methodName, boolean isOneway) { - return new fake_proc<>(methodName, isOneway); - } - - @Test - public void testSomeOnewayMethods() { - Map> 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 onewayMethods = RpcWrapper.getOnewayMethods(procs); - Assert.assertEquals(Sets.newHashSet("foo", "bar"), onewayMethods); - } - - @Test - public void testNoOnewayMethods() { - Map> 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 onewayMethods = RpcWrapper.getOnewayMethods(procs); - Assert.assertEquals(Collections. emptySet(), onewayMethods); - } - - @Test - public void testAllOnewayMethods() { - Map> 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 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 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 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 - extends org.apache.thrift.ProcessFunction { - 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 { - - 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 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(clientHandler)); + Iface rpcProxy = RpcWrapper.service(clientHandler); Processor 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(this)); + Iface rpcProxy = RpcWrapper.service(this); final Processor 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(clientHandler)); + Iface rpcProxy = RpcWrapper.service(clientHandler); final Processor processor; if (ThriftServerType.SASL == getThriftServerType()) { Iface tcredsProxy = TCredentialsUpdatingWrapper.service(rpcProxy, clientHandler.getClass(), @@ -1358,8 +1358,7 @@ public class Master extends AccumuloServerContext ReplicationCoordinator.Processor replicationCoordinatorProcessor = new ReplicationCoordinator.Processor<>( // @formatter:on - RpcWrapper.service(impl, - new ReplicationCoordinator.Processor(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(clientHandler)); + Iface rpcProxy = RpcWrapper.service(clientHandler); final Processor 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(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 @@ + + thrift + + + + org.codehaus.mojo + exec-maven-plugin + + + generate-thrift + + exec + + generate-sources + + ${basedir}/src/main/scripts/generate-thrift.sh + + + + + + + mrit 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. --> + + + + + + 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 + *

+ * 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 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. 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 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 resultHandler) throws org.apache.thrift.TException; + + public void onewayPass(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void echoFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void onewayFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void echoRuntimeFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void onewayRuntimeFail(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback 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 { + 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 { + 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 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 { + private java.lang.String value; + public echoPass_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback 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 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 { + private java.lang.String value; + public onewayPass_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback 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 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 { + private java.lang.String value; + public echoFail_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback 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 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 { + private java.lang.String value; + public onewayFail_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback 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 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 { + private java.lang.String value; + public echoRuntimeFail_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback 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 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 { + private java.lang.String value; + public onewayRuntimeFail_call(java.lang.String value, org.apache.thrift.async.AsyncMethodCallback 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 extends org.apache.thrift.TBaseProcessor 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>())); + } + + protected Processor(I iface, java.util.Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static java.util.Map> getProcessMap(java.util.Map> 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 extends org.apache.thrift.ProcessFunction { + 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 extends org.apache.thrift.ProcessFunction { + 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 extends org.apache.thrift.ProcessFunction { + 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 extends org.apache.thrift.ProcessFunction { + 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 extends org.apache.thrift.ProcessFunction { + 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 extends org.apache.thrift.ProcessFunction { + 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 extends org.apache.thrift.TBaseAsyncProcessor { + 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>())); + } + + protected AsyncProcessor(I iface, java.util.Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static java.util.Map> getProcessMap(java.util.Map> 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 extends org.apache.thrift.AsyncProcessFunction { + public echoPass() { + super("echoPass"); + } + + public echoPass_args getEmptyArgsInstance() { + return new echoPass_args(); + } + + public org.apache.thrift.async.AsyncMethodCallback 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() { + 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 resultHandler) throws org.apache.thrift.TException { + iface.echoPass(args.value,resultHandler); + } + } + + public static class onewayPass extends org.apache.thrift.AsyncProcessFunction { + public onewayPass() { + super("onewayPass"); + } + + public onewayPass_args getEmptyArgsInstance() { + return new onewayPass_args(); + } + + public org.apache.thrift.async.AsyncMethodCallback 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() { + 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 resultHandler) throws org.apache.thrift.TException { + iface.onewayPass(args.value,resultHandler); + } + } + + public static class echoFail extends org.apache.thrift.AsyncProcessFunction { + public echoFail() { + super("echoFail"); + } + + public echoFail_args getEmptyArgsInstance() { + return new echoFail_args(); + } + + public org.apache.thrift.async.AsyncMethodCallback 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() { + 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 resultHandler) throws org.apache.thrift.TException { + iface.echoFail(args.value,resultHandler); + } + } + + public static class onewayFail extends org.apache.thrift.AsyncProcessFunction { + public onewayFail() { + super("onewayFail"); + } + + public onewayFail_args getEmptyArgsInstance() { + return new onewayFail_args(); + } + + public org.apache.thrift.async.AsyncMethodCallback 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() { + 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 resultHandler) throws org.apache.thrift.TException { + iface.onewayFail(args.value,resultHandler); + } + } + + public static class echoRuntimeFail extends org.apache.thrift.AsyncProcessFunction { + public echoRuntimeFail() { + super("echoRuntimeFail"); + } + + public echoRuntimeFail_args getEmptyArgsInstance() { + return new echoRuntimeFail_args(); + } + + public org.apache.thrift.async.AsyncMethodCallback 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() { + 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 resultHandler) throws org.apache.thrift.TException { + iface.echoRuntimeFail(args.value,resultHandler); + } + } + + public static class onewayRuntimeFail extends org.apache.thrift.AsyncProcessFunction { + public onewayRuntimeFail() { + super("onewayRuntimeFail"); + } + + public onewayRuntimeFail_args getEmptyArgsInstance() { + return new onewayRuntimeFail_args(); + } + + public org.apache.thrift.async.AsyncMethodCallback 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() { + 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 resultHandler) throws org.apache.thrift.TException { + iface.onewayRuntimeFail(args.value,resultHandler); + } + } + + } + + public static class echoPass_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + 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 byName = new java.util.HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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 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, java.io.Serializable, Cloneable, Comparable { + 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 byName = new java.util.HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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 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, java.io.Serializable, Cloneable, Comparable { + 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 byName = new java.util.HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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 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, java.io.Serializable, Cloneable, Comparable { + 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 byName = new java.util.HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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 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, java.io.Serializable, Cloneable, Comparable { + 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 byName = new java.util.HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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 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, java.io.Serializable, Cloneable, Comparable { + 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 byName = new java.util.HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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 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, java.io.Serializable, Cloneable, Comparable { + 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 byName = new java.util.HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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 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, java.io.Serializable, Cloneable, Comparable { + 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 byName = new java.util.HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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 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, java.io.Serializable, Cloneable, Comparable { + 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 byName = new java.util.HashMap(); + + 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 other. + */ + 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 { + + 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 { + + @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 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) +}