mirror of https://github.com/apache/activemq.git
Adding a new OpenWire command called BrokerSubscriptionInfo in order to help synchronize durable subs across a network bridge. Added OpenWire version 12. For dynamicallyIncludedDestination durable subs will now be synchronized on a bridge reconnect as long as the bridge supports conduitSubscriptions and dynamicOnly=false
This commit is contained in:
parent
a65f5e7c20
commit
3953b9aaef
|
@ -20,6 +20,7 @@ import java.io.EOFException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.SocketException;
|
import java.net.SocketException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -27,7 +28,6 @@ import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
import java.util.Set;
|
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
@ -41,9 +41,12 @@ import javax.transaction.xa.XAResource;
|
||||||
|
|
||||||
import org.apache.activemq.advisory.AdvisorySupport;
|
import org.apache.activemq.advisory.AdvisorySupport;
|
||||||
import org.apache.activemq.broker.region.ConnectionStatistics;
|
import org.apache.activemq.broker.region.ConnectionStatistics;
|
||||||
|
import org.apache.activemq.broker.region.DurableTopicSubscription;
|
||||||
import org.apache.activemq.broker.region.RegionBroker;
|
import org.apache.activemq.broker.region.RegionBroker;
|
||||||
|
import org.apache.activemq.broker.region.TopicRegion;
|
||||||
import org.apache.activemq.command.ActiveMQDestination;
|
import org.apache.activemq.command.ActiveMQDestination;
|
||||||
import org.apache.activemq.command.BrokerInfo;
|
import org.apache.activemq.command.BrokerInfo;
|
||||||
|
import org.apache.activemq.command.BrokerSubscriptionInfo;
|
||||||
import org.apache.activemq.command.Command;
|
import org.apache.activemq.command.Command;
|
||||||
import org.apache.activemq.command.CommandTypes;
|
import org.apache.activemq.command.CommandTypes;
|
||||||
import org.apache.activemq.command.ConnectionControl;
|
import org.apache.activemq.command.ConnectionControl;
|
||||||
|
@ -100,6 +103,7 @@ import org.apache.activemq.transport.Transport;
|
||||||
import org.apache.activemq.transport.TransportDisposedIOException;
|
import org.apache.activemq.transport.TransportDisposedIOException;
|
||||||
import org.apache.activemq.util.IntrospectionSupport;
|
import org.apache.activemq.util.IntrospectionSupport;
|
||||||
import org.apache.activemq.util.MarshallingSupport;
|
import org.apache.activemq.util.MarshallingSupport;
|
||||||
|
import org.apache.activemq.util.SubscriptionKey;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
import org.slf4j.MDC;
|
import org.slf4j.MDC;
|
||||||
|
@ -1373,20 +1377,58 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
this.pendingStop = pendingStop;
|
this.pendingStop = pendingStop;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
public static BrokerSubscriptionInfo getBrokerSubscriptionInfo(final BrokerService brokerService) {
|
||||||
public Response processBrokerInfo(BrokerInfo info) {
|
RegionBroker regionBroker = (RegionBroker) brokerService.getRegionBroker();
|
||||||
if (info.isSlaveBroker()) {
|
TopicRegion topicRegion = (TopicRegion) regionBroker.getTopicRegion();
|
||||||
LOG.error(" Slave Brokers are no longer supported - slave trying to attach is: {}", info.getBrokerName());
|
List<ConsumerInfo> subscriptionInfos = new ArrayList<>();
|
||||||
} else if (info.isNetworkConnection() && info.isDuplexConnection()) {
|
for (SubscriptionKey key : topicRegion.getDurableSubscriptions().keySet()) {
|
||||||
// so this TransportConnection is the rear end of a network bridge
|
DurableTopicSubscription sub = topicRegion.getDurableSubscriptions().get(key);
|
||||||
// We have been requested to create a two way pipe ...
|
if (sub != null) {
|
||||||
try {
|
ConsumerInfo ci = sub.getConsumerInfo().copy();
|
||||||
|
ci.setClientId(key.getClientId());
|
||||||
|
subscriptionInfos.add(ci);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
BrokerSubscriptionInfo bsi = new BrokerSubscriptionInfo(brokerService.getBrokerName());
|
||||||
|
bsi.setSubscriptionInfos(subscriptionInfos.toArray(new ConsumerInfo[0]));
|
||||||
|
return bsi;
|
||||||
|
}
|
||||||
|
|
||||||
|
private NetworkBridgeConfiguration getNetworkConfiguration(final BrokerInfo info) throws IOException {
|
||||||
Properties properties = MarshallingSupport.stringToProperties(info.getNetworkProperties());
|
Properties properties = MarshallingSupport.stringToProperties(info.getNetworkProperties());
|
||||||
Map<String, String> props = createMap(properties);
|
Map<String, String> props = createMap(properties);
|
||||||
NetworkBridgeConfiguration config = new NetworkBridgeConfiguration();
|
NetworkBridgeConfiguration config = new NetworkBridgeConfiguration();
|
||||||
IntrospectionSupport.setProperties(config, props, "");
|
IntrospectionSupport.setProperties(config, props, "");
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Response processBrokerInfo(BrokerInfo info) {
|
||||||
|
if (info.isSlaveBroker()) {
|
||||||
|
LOG.error(" Slave Brokers are no longer supported - slave trying to attach is: {}", info.getBrokerName());
|
||||||
|
} else if (info.isNetworkConnection() && !info.isDuplexConnection()) {
|
||||||
|
try {
|
||||||
|
NetworkBridgeConfiguration config = getNetworkConfiguration(info);
|
||||||
|
if (config.isSyncDurableSubs() && protocolVersion.get() >= CommandTypes.PROTOCOL_VERSION_DURABLE_SYNC) {
|
||||||
|
LOG.debug("SyncDurableSubs is enabled, Sending BrokerSubscriptionInfo");
|
||||||
|
dispatchSync(getBrokerSubscriptionInfo(this.broker.getBrokerService()));
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("Failed to respond to network bridge creation from broker {}", info.getBrokerId(), e);
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
} else if (info.isNetworkConnection() && info.isDuplexConnection()) {
|
||||||
|
// so this TransportConnection is the rear end of a network bridge
|
||||||
|
// We have been requested to create a two way pipe ...
|
||||||
|
try {
|
||||||
|
NetworkBridgeConfiguration config = getNetworkConfiguration(info);
|
||||||
config.setBrokerName(broker.getBrokerName());
|
config.setBrokerName(broker.getBrokerName());
|
||||||
|
|
||||||
|
if (config.isSyncDurableSubs() && protocolVersion.get() >= 12) {
|
||||||
|
LOG.debug("SyncDurableSubs is enabled, Sending BrokerSubscriptionInfo");
|
||||||
|
dispatchSync(getBrokerSubscriptionInfo(this.broker.getBrokerService()));
|
||||||
|
}
|
||||||
|
|
||||||
// check for existing duplex connection hanging about
|
// check for existing duplex connection hanging about
|
||||||
|
|
||||||
// We first look if existing network connection already exists for the same broker Id and network connector name
|
// We first look if existing network connection already exists for the same broker Id and network connector name
|
||||||
|
@ -1698,4 +1740,12 @@ public class TransportConnection implements Connection, Task, CommandVisitor {
|
||||||
public WireFormatInfo getRemoteWireFormatInfo() {
|
public WireFormatInfo getRemoteWireFormatInfo() {
|
||||||
return wireFormatInfo;
|
return wireFormatInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/* (non-Javadoc)
|
||||||
|
* @see org.apache.activemq.state.CommandVisitor#processBrokerSubscriptionInfo(org.apache.activemq.command.BrokerSubscriptionInfo)
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Response processBrokerSubscriptionInfo(BrokerSubscriptionInfo info) throws Exception {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -57,7 +57,9 @@ import org.apache.activemq.command.ActiveMQTempDestination;
|
||||||
import org.apache.activemq.command.ActiveMQTopic;
|
import org.apache.activemq.command.ActiveMQTopic;
|
||||||
import org.apache.activemq.command.BrokerId;
|
import org.apache.activemq.command.BrokerId;
|
||||||
import org.apache.activemq.command.BrokerInfo;
|
import org.apache.activemq.command.BrokerInfo;
|
||||||
|
import org.apache.activemq.command.BrokerSubscriptionInfo;
|
||||||
import org.apache.activemq.command.Command;
|
import org.apache.activemq.command.Command;
|
||||||
|
import org.apache.activemq.command.CommandTypes;
|
||||||
import org.apache.activemq.command.ConnectionError;
|
import org.apache.activemq.command.ConnectionError;
|
||||||
import org.apache.activemq.command.ConnectionId;
|
import org.apache.activemq.command.ConnectionId;
|
||||||
import org.apache.activemq.command.ConnectionInfo;
|
import org.apache.activemq.command.ConnectionInfo;
|
||||||
|
@ -127,11 +129,12 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
protected ActiveMQDestination[] dynamicallyIncludedDestinations;
|
protected ActiveMQDestination[] dynamicallyIncludedDestinations;
|
||||||
protected ActiveMQDestination[] staticallyIncludedDestinations;
|
protected ActiveMQDestination[] staticallyIncludedDestinations;
|
||||||
protected ActiveMQDestination[] durableDestinations;
|
protected ActiveMQDestination[] durableDestinations;
|
||||||
protected final ConcurrentMap<ConsumerId, DemandSubscription> subscriptionMapByLocalId = new ConcurrentHashMap<ConsumerId, DemandSubscription>();
|
protected final ConcurrentMap<ConsumerId, DemandSubscription> subscriptionMapByLocalId = new ConcurrentHashMap<>();
|
||||||
protected final ConcurrentMap<ConsumerId, DemandSubscription> subscriptionMapByRemoteId = new ConcurrentHashMap<ConsumerId, DemandSubscription>();
|
protected final ConcurrentMap<ConsumerId, DemandSubscription> subscriptionMapByRemoteId = new ConcurrentHashMap<>();
|
||||||
protected final BrokerId localBrokerPath[] = new BrokerId[]{null};
|
protected final BrokerId localBrokerPath[] = new BrokerId[]{null};
|
||||||
protected final CountDownLatch startedLatch = new CountDownLatch(2);
|
protected final CountDownLatch startedLatch = new CountDownLatch(2);
|
||||||
protected final CountDownLatch localStartedLatch = new CountDownLatch(1);
|
protected final CountDownLatch localStartedLatch = new CountDownLatch(1);
|
||||||
|
protected final CountDownLatch staticDestinationsLatch = new CountDownLatch(1);
|
||||||
protected final AtomicBoolean lastConnectSucceeded = new AtomicBoolean(false);
|
protected final AtomicBoolean lastConnectSucceeded = new AtomicBoolean(false);
|
||||||
protected NetworkBridgeConfiguration configuration;
|
protected NetworkBridgeConfiguration configuration;
|
||||||
protected final NetworkBridgeFilterFactory defaultFilterFactory = new DefaultNetworkBridgeFilterFactory();
|
protected final NetworkBridgeFilterFactory defaultFilterFactory = new DefaultNetworkBridgeFilterFactory();
|
||||||
|
@ -311,6 +314,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
startedLatch.countDown();
|
startedLatch.countDown();
|
||||||
startedLatch.countDown();
|
startedLatch.countDown();
|
||||||
localStartedLatch.countDown();
|
localStartedLatch.countDown();
|
||||||
|
staticDestinationsLatch.countDown();
|
||||||
|
|
||||||
ss.throwFirstException();
|
ss.throwFirstException();
|
||||||
}
|
}
|
||||||
|
@ -440,6 +444,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
try {
|
try {
|
||||||
if (safeWaitUntilStarted()) {
|
if (safeWaitUntilStarted()) {
|
||||||
setupStaticDestinations();
|
setupStaticDestinations();
|
||||||
|
staticDestinationsLatch.countDown();
|
||||||
}
|
}
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
serviceLocalException(e);
|
serviceLocalException(e);
|
||||||
|
@ -549,6 +554,10 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
brokerInfo.setNetworkProperties(str);
|
brokerInfo.setNetworkProperties(str);
|
||||||
brokerInfo.setBrokerId(this.localBrokerId);
|
brokerInfo.setBrokerId(this.localBrokerId);
|
||||||
remoteBroker.oneway(brokerInfo);
|
remoteBroker.oneway(brokerInfo);
|
||||||
|
if (configuration.isSyncDurableSubs() &&
|
||||||
|
remoteBroker.getWireFormat().getVersion() >= CommandTypes.PROTOCOL_VERSION_DURABLE_SYNC) {
|
||||||
|
remoteBroker.oneway(TransportConnection.getBrokerSubscriptionInfo(brokerService));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
if (remoteConnectionInfo != null) {
|
if (remoteConnectionInfo != null) {
|
||||||
remoteBroker.oneway(remoteConnectionInfo.createRemoveCommand());
|
remoteBroker.oneway(remoteConnectionInfo.createRemoveCommand());
|
||||||
|
@ -617,6 +626,31 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
ackAdvisory(md.getMessage());
|
ackAdvisory(md.getMessage());
|
||||||
} else if (command.isBrokerInfo()) {
|
} else if (command.isBrokerInfo()) {
|
||||||
futureRemoteBrokerInfo.set((BrokerInfo) command);
|
futureRemoteBrokerInfo.set((BrokerInfo) command);
|
||||||
|
} else if (command instanceof BrokerSubscriptionInfo) {
|
||||||
|
staticDestinationsLatch.await();
|
||||||
|
BrokerSubscriptionInfo subInfo = (BrokerSubscriptionInfo) command;
|
||||||
|
LOG.debug("Received Remote BrokerSubscriptionInfo on {} from {}",
|
||||||
|
this.brokerService.getBrokerName(), subInfo.getBrokerName());
|
||||||
|
|
||||||
|
if (configuration.isSyncDurableSubs() && configuration.isConduitSubscriptions()
|
||||||
|
&& !configuration.isDynamicOnly() && subInfo.getSubscriptionInfos() != null) {
|
||||||
|
if (started.get()) {
|
||||||
|
for (ConsumerInfo info : subInfo.getSubscriptionInfos()) {
|
||||||
|
if(!info.getSubscriptionName().startsWith(DURABLE_SUB_PREFIX) &&
|
||||||
|
matchesDynamicallyIncludedDestinations(info.getDestination())) {
|
||||||
|
serviceRemoteConsumerAdvisory(info);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
//After re-added, clean up any empty durables
|
||||||
|
for (Iterator<DemandSubscription> i = subscriptionMapByLocalId.values().iterator(); i.hasNext(); ) {
|
||||||
|
DemandSubscription ds = i.next();
|
||||||
|
if (matchesDynamicallyIncludedDestinations(ds.getLocalInfo().getDestination())) {
|
||||||
|
cleanupDurableSub(ds, i);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
} else if (command.getClass() == ConnectionError.class) {
|
} else if (command.getClass() == ConnectionError.class) {
|
||||||
ConnectionError ce = (ConnectionError) command;
|
ConnectionError ce = (ConnectionError) command;
|
||||||
serviceRemoteException(ce.getException());
|
serviceRemoteException(ce.getException());
|
||||||
|
@ -831,7 +865,15 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
DemandSubscription ds = i.next();
|
DemandSubscription ds = i.next();
|
||||||
boolean removed = ds.getDurableRemoteSubs().remove(subscriptionInfo);
|
boolean removed = ds.getDurableRemoteSubs().remove(subscriptionInfo);
|
||||||
if (removed) {
|
if (removed) {
|
||||||
if (ds.getDurableRemoteSubs().isEmpty()) {
|
cleanupDurableSub(ds, i);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void cleanupDurableSub(final DemandSubscription ds,
|
||||||
|
Iterator<DemandSubscription> i) throws IOException {
|
||||||
|
if (ds != null && ds.getLocalDurableSubscriber() != null && ds.getDurableRemoteSubs().isEmpty()) {
|
||||||
|
|
||||||
// deactivate subscriber
|
// deactivate subscriber
|
||||||
RemoveInfo removeInfo = new RemoveInfo(ds.getLocalInfo().getConsumerId());
|
RemoveInfo removeInfo = new RemoveInfo(ds.getLocalInfo().getConsumerId());
|
||||||
|
@ -848,9 +890,6 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
i.remove();
|
i.remove();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serviceLocalException(Throwable error) {
|
public void serviceLocalException(Throwable error) {
|
||||||
|
@ -1064,6 +1103,8 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
switch (command.getDataStructureType()) {
|
switch (command.getDataStructureType()) {
|
||||||
case WireFormatInfo.DATA_STRUCTURE_TYPE:
|
case WireFormatInfo.DATA_STRUCTURE_TYPE:
|
||||||
break;
|
break;
|
||||||
|
case BrokerSubscriptionInfo.DATA_STRUCTURE_TYPE:
|
||||||
|
break;
|
||||||
default:
|
default:
|
||||||
LOG.warn("Unexpected local command: {}", command);
|
LOG.warn("Unexpected local command: {}", command);
|
||||||
}
|
}
|
||||||
|
@ -1154,6 +1195,15 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
}
|
}
|
||||||
|
|
||||||
dests = dynamicallyIncludedDestinations;
|
dests = dynamicallyIncludedDestinations;
|
||||||
|
if (dests != null && dests.length > 0) {
|
||||||
|
return matchesDynamicallyIncludedDestinations(destination);
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean matchesDynamicallyIncludedDestinations(ActiveMQDestination destination) {
|
||||||
|
ActiveMQDestination[] dests = dynamicallyIncludedDestinations;
|
||||||
if (dests != null && dests.length > 0) {
|
if (dests != null && dests.length > 0) {
|
||||||
for (ActiveMQDestination dest : dests) {
|
for (ActiveMQDestination dest : dests) {
|
||||||
DestinationFilter inclusionFilter = DestinationFilter.parseFilter(dest);
|
DestinationFilter inclusionFilter = DestinationFilter.parseFilter(dest);
|
||||||
|
@ -1161,11 +1211,10 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Subscriptions for these destinations are always created
|
* Subscriptions for these destinations are always created
|
||||||
|
@ -1175,7 +1224,7 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
if (dests != null) {
|
if (dests != null) {
|
||||||
for (ActiveMQDestination dest : dests) {
|
for (ActiveMQDestination dest : dests) {
|
||||||
if (isPermissableDestination(dest)) {
|
if (isPermissableDestination(dest)) {
|
||||||
DemandSubscription sub = createDemandSubscription(dest);
|
DemandSubscription sub = createDemandSubscription(dest, null);
|
||||||
sub.setStaticallyIncluded(true);
|
sub.setStaticallyIncluded(true);
|
||||||
try {
|
try {
|
||||||
addSubscription(sub);
|
addSubscription(sub);
|
||||||
|
@ -1348,11 +1397,15 @@ public abstract class DemandForwardingBridgeSupport implements NetworkBridge, Br
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
final protected DemandSubscription createDemandSubscription(ActiveMQDestination destination) {
|
final protected DemandSubscription createDemandSubscription(ActiveMQDestination destination, final String subscriptionName) {
|
||||||
ConsumerInfo info = new ConsumerInfo();
|
ConsumerInfo info = new ConsumerInfo();
|
||||||
info.setNetworkSubscription(true);
|
info.setNetworkSubscription(true);
|
||||||
info.setDestination(destination);
|
info.setDestination(destination);
|
||||||
|
|
||||||
|
if (subscriptionName != null) {
|
||||||
|
info.setSubscriptionName(subscriptionName);
|
||||||
|
}
|
||||||
|
|
||||||
// Indicate that this subscription is being made on behalf of the remote broker.
|
// Indicate that this subscription is being made on behalf of the remote broker.
|
||||||
info.setBrokerPath(new BrokerId[]{remoteBrokerId});
|
info.setBrokerPath(new BrokerId[]{remoteBrokerId});
|
||||||
|
|
||||||
|
|
|
@ -73,7 +73,7 @@ public class DurableConduitBridge extends ConduitBridge {
|
||||||
for (Subscription subscription : topicRegion.getDurableSubscriptions().values()) {
|
for (Subscription subscription : topicRegion.getDurableSubscriptions().values()) {
|
||||||
String subName = subscription.getConsumerInfo().getSubscriptionName();
|
String subName = subscription.getConsumerInfo().getSubscriptionName();
|
||||||
if (subName != null && subName.equals(candidateSubName)) {
|
if (subName != null && subName.equals(candidateSubName)) {
|
||||||
DemandSubscription sub = createDemandSubscription(dest);
|
DemandSubscription sub = createDemandSubscription(dest, subName);
|
||||||
sub.getLocalInfo().setSubscriptionName(getSubscriberName(dest));
|
sub.getLocalInfo().setSubscriptionName(getSubscriberName(dest));
|
||||||
sub.setStaticallyIncluded(true);
|
sub.setStaticallyIncluded(true);
|
||||||
addSubscription(sub);
|
addSubscription(sub);
|
||||||
|
|
|
@ -31,6 +31,7 @@ public class NetworkBridgeConfiguration {
|
||||||
private boolean conduitSubscriptions = true;
|
private boolean conduitSubscriptions = true;
|
||||||
private boolean useVirtualDestSubs;
|
private boolean useVirtualDestSubs;
|
||||||
private boolean dynamicOnly;
|
private boolean dynamicOnly;
|
||||||
|
private boolean syncDurableSubs;
|
||||||
private boolean dispatchAsync = true;
|
private boolean dispatchAsync = true;
|
||||||
private boolean decreaseNetworkConsumerPriority;
|
private boolean decreaseNetworkConsumerPriority;
|
||||||
private int consumerPriorityBase = ConsumerInfo.NETWORK_CONSUMER_PRIORITY;
|
private int consumerPriorityBase = ConsumerInfo.NETWORK_CONSUMER_PRIORITY;
|
||||||
|
@ -98,6 +99,14 @@ public class NetworkBridgeConfiguration {
|
||||||
this.dynamicOnly = dynamicOnly;
|
this.dynamicOnly = dynamicOnly;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean isSyncDurableSubs() {
|
||||||
|
return syncDurableSubs;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSyncDurableSubs(boolean syncDurableSubs) {
|
||||||
|
this.syncDurableSubs = syncDurableSubs;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return the bridgeTempDestinations
|
* @return the bridgeTempDestinations
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -308,7 +308,7 @@
|
||||||
<tasks>
|
<tasks>
|
||||||
<echo>Running OpenWire Generator</echo>
|
<echo>Running OpenWire Generator</echo>
|
||||||
<taskdef name="generate" classname="org.apache.activemq.openwire.tool.JavaGeneratorTask" classpathref="maven.compile.classpath" />
|
<taskdef name="generate" classname="org.apache.activemq.openwire.tool.JavaGeneratorTask" classpathref="maven.compile.classpath" />
|
||||||
<generate version="11" basedir="${basedir}" generateTests="false" />
|
<generate version="12" basedir="${basedir}" generateTests="false" />
|
||||||
</tasks>
|
</tasks>
|
||||||
</configuration>
|
</configuration>
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -74,6 +74,7 @@ public class BrokerInfo extends BaseCommand {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public byte getDataStructureType() {
|
public byte getDataStructureType() {
|
||||||
return DATA_STRUCTURE_TYPE;
|
return DATA_STRUCTURE_TYPE;
|
||||||
}
|
}
|
||||||
|
@ -122,6 +123,7 @@ public class BrokerInfo extends BaseCommand {
|
||||||
this.brokerName = brokerName;
|
this.brokerName = brokerName;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response visit(CommandVisitor visitor) throws Exception {
|
public Response visit(CommandVisitor visitor) throws Exception {
|
||||||
return visitor.processBrokerInfo(this);
|
return visitor.processBrokerInfo(this);
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,99 @@
|
||||||
|
/**
|
||||||
|
* 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.activemq.command;
|
||||||
|
|
||||||
|
import org.apache.activemq.state.CommandVisitor;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Used to represent a durable subscription.
|
||||||
|
*
|
||||||
|
* @openwire:marshaller code="92"
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class BrokerSubscriptionInfo extends BaseCommand {
|
||||||
|
|
||||||
|
public static final byte DATA_STRUCTURE_TYPE = CommandTypes.BROKER_SUBSCRIPTION_INFO;
|
||||||
|
|
||||||
|
BrokerId brokerId;
|
||||||
|
String brokerName;
|
||||||
|
ConsumerInfo subscriptionInfos[];
|
||||||
|
|
||||||
|
public BrokerSubscriptionInfo() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public BrokerSubscriptionInfo(String brokerName) {
|
||||||
|
this.brokerName = brokerName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public BrokerSubscriptionInfo copy() {
|
||||||
|
BrokerSubscriptionInfo copy = new BrokerSubscriptionInfo();
|
||||||
|
copy(copy);
|
||||||
|
return copy;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void copy(BrokerSubscriptionInfo copy) {
|
||||||
|
super.copy(copy);
|
||||||
|
copy.subscriptionInfos = this.subscriptionInfos;
|
||||||
|
copy.brokerName = this.brokerName;
|
||||||
|
copy.brokerId = this.brokerId;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Response visit(CommandVisitor visitor) throws Exception {
|
||||||
|
return visitor.processBrokerSubscriptionInfo(this);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @openwire:property version=12
|
||||||
|
*/
|
||||||
|
public BrokerId getBrokerId() {
|
||||||
|
return brokerId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setBrokerId(BrokerId brokerId) {
|
||||||
|
this.brokerId = brokerId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @openwire:property version=12
|
||||||
|
*/
|
||||||
|
public String getBrokerName() {
|
||||||
|
return brokerName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setBrokerName(String brokerName) {
|
||||||
|
this.brokerName = brokerName;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @openwire:property version=12
|
||||||
|
*/
|
||||||
|
public ConsumerInfo[] getSubscriptionInfos() {
|
||||||
|
return subscriptionInfos;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setSubscriptionInfos(ConsumerInfo[] subscriptionInfos) {
|
||||||
|
this.subscriptionInfos = subscriptionInfos;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -24,7 +24,7 @@ package org.apache.activemq.command;
|
||||||
public interface CommandTypes {
|
public interface CommandTypes {
|
||||||
|
|
||||||
// What is the latest version of the openwire protocol
|
// What is the latest version of the openwire protocol
|
||||||
byte PROTOCOL_VERSION = 11;
|
byte PROTOCOL_VERSION = 12;
|
||||||
|
|
||||||
// What is the latest version of the openwire protocol used in the stores
|
// What is the latest version of the openwire protocol used in the stores
|
||||||
byte PROTOCOL_STORE_VERSION = 11;
|
byte PROTOCOL_STORE_VERSION = 11;
|
||||||
|
@ -32,6 +32,9 @@ public interface CommandTypes {
|
||||||
// What is the legacy version that old KahaDB store's most commonly used
|
// What is the legacy version that old KahaDB store's most commonly used
|
||||||
byte PROTOCOL_LEGACY_STORE_VERSION = 6;
|
byte PROTOCOL_LEGACY_STORE_VERSION = 6;
|
||||||
|
|
||||||
|
// What is the first version that BROKER_SUBSCRIPTION_INFO is supported
|
||||||
|
byte PROTOCOL_VERSION_DURABLE_SYNC = 12;
|
||||||
|
|
||||||
// A marshaling layer can use this type to specify a null object.
|
// A marshaling layer can use this type to specify a null object.
|
||||||
byte NULL = 0;
|
byte NULL = 0;
|
||||||
|
|
||||||
|
@ -93,7 +96,7 @@ public interface CommandTypes {
|
||||||
//
|
//
|
||||||
// Used by discovery
|
// Used by discovery
|
||||||
//
|
//
|
||||||
// /////////////////////////////////////////////////
|
// /////////////////////////////////////////////////BROKER_SUBSCRIPTION_INFO
|
||||||
byte DISCOVERY_EVENT = 40;
|
byte DISCOVERY_EVENT = 40;
|
||||||
|
|
||||||
// /////////////////////////////////////////////////
|
// /////////////////////////////////////////////////
|
||||||
|
@ -141,6 +144,7 @@ public interface CommandTypes {
|
||||||
|
|
||||||
byte MESSAGE_DISPATCH_NOTIFICATION = 90;
|
byte MESSAGE_DISPATCH_NOTIFICATION = 90;
|
||||||
byte NETWORK_BRIDGE_FILTER = 91;
|
byte NETWORK_BRIDGE_FILTER = 91;
|
||||||
|
byte BROKER_SUBSCRIPTION_INFO = 92;
|
||||||
|
|
||||||
// /////////////////////////////////////////////////
|
// /////////////////////////////////////////////////
|
||||||
//
|
//
|
||||||
|
|
|
@ -0,0 +1,139 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQBlobMessageMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQBlobMessageMarshaller extends ActiveMQMessageMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQBlobMessage.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQBlobMessage();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ActiveMQBlobMessage info = (ActiveMQBlobMessage)o;
|
||||||
|
info.setRemoteBlobUrl(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setMimeType(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setDeletedByBroker(bs.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ActiveMQBlobMessage info = (ActiveMQBlobMessage)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getRemoteBlobUrl(), bs);
|
||||||
|
rc += tightMarshalString1(info.getMimeType(), bs);
|
||||||
|
bs.writeBoolean(info.isDeletedByBroker());
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ActiveMQBlobMessage info = (ActiveMQBlobMessage)o;
|
||||||
|
tightMarshalString2(info.getRemoteBlobUrl(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getMimeType(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ActiveMQBlobMessage info = (ActiveMQBlobMessage)o;
|
||||||
|
info.setRemoteBlobUrl(looseUnmarshalString(dataIn));
|
||||||
|
info.setMimeType(looseUnmarshalString(dataIn));
|
||||||
|
info.setDeletedByBroker(dataIn.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ActiveMQBlobMessage info = (ActiveMQBlobMessage)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getRemoteBlobUrl(), dataOut);
|
||||||
|
looseMarshalString(info.getMimeType(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isDeletedByBroker());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQBytesMessageMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQBytesMessageMarshaller extends ActiveMQMessageMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQBytesMessage.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQBytesMessage();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQDestinationMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public abstract class ActiveMQDestinationMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ActiveMQDestination info = (ActiveMQDestination)o;
|
||||||
|
info.setPhysicalName(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ActiveMQDestination info = (ActiveMQDestination)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getPhysicalName(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ActiveMQDestination info = (ActiveMQDestination)o;
|
||||||
|
tightMarshalString2(info.getPhysicalName(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ActiveMQDestination info = (ActiveMQDestination)o;
|
||||||
|
info.setPhysicalName(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ActiveMQDestination info = (ActiveMQDestination)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getPhysicalName(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQMapMessageMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQMapMessageMarshaller extends ActiveMQMessageMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQMapMessage.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQMapMessage();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQMessageMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQMessageMarshaller extends MessageMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQMessage.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQMessage();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQObjectMessageMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQObjectMessageMarshaller extends ActiveMQMessageMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQObjectMessage.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQObjectMessage();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQQueueMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQQueueMarshaller extends ActiveMQDestinationMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQQueue.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQQueue();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQStreamMessageMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQStreamMessageMarshaller extends ActiveMQMessageMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQStreamMessage.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQStreamMessage();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,99 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQTempDestinationMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public abstract class ActiveMQTempDestinationMarshaller extends ActiveMQDestinationMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQTempQueueMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQTempQueueMarshaller extends ActiveMQTempDestinationMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQTempQueue.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQTempQueue();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQTempTopicMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQTempTopicMarshaller extends ActiveMQTempDestinationMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQTempTopic.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQTempTopic();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQTextMessageMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQTextMessageMarshaller extends ActiveMQMessageMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQTextMessage.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQTextMessage();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ActiveMQTopicMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ActiveMQTopicMarshaller extends ActiveMQDestinationMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ActiveMQTopic.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ActiveMQTopic();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,118 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for BaseCommandMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public abstract class BaseCommandMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
BaseCommand info = (BaseCommand)o;
|
||||||
|
info.setCommandId(dataIn.readInt());
|
||||||
|
info.setResponseRequired(bs.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
BaseCommand info = (BaseCommand)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
bs.writeBoolean(info.isResponseRequired());
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
BaseCommand info = (BaseCommand)o;
|
||||||
|
dataOut.writeInt(info.getCommandId());
|
||||||
|
bs.readBoolean();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
BaseCommand info = (BaseCommand)o;
|
||||||
|
info.setCommandId(dataIn.readInt());
|
||||||
|
info.setResponseRequired(dataIn.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
BaseCommand info = (BaseCommand)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
dataOut.writeInt(info.getCommandId());
|
||||||
|
dataOut.writeBoolean(info.isResponseRequired());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,644 @@
|
||||||
|
/**
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.lang.reflect.Constructor;
|
||||||
|
import org.apache.activemq.command.DataStructure;
|
||||||
|
import org.apache.activemq.openwire.BooleanStream;
|
||||||
|
import org.apache.activemq.openwire.DataStreamMarshaller;
|
||||||
|
import org.apache.activemq.openwire.OpenWireFormat;
|
||||||
|
import org.apache.activemq.util.ByteSequence;
|
||||||
|
|
||||||
|
public abstract class BaseDataStreamMarshaller implements DataStreamMarshaller {
|
||||||
|
|
||||||
|
public static final Constructor STACK_TRACE_ELEMENT_CONSTRUCTOR;
|
||||||
|
|
||||||
|
static {
|
||||||
|
Constructor constructor = null;
|
||||||
|
try {
|
||||||
|
constructor = StackTraceElement.class.getConstructor(new Class[] {String.class, String.class,
|
||||||
|
String.class, int.class});
|
||||||
|
} catch (Throwable e) {
|
||||||
|
}
|
||||||
|
STACK_TRACE_ELEMENT_CONSTRUCTOR = constructor;
|
||||||
|
}
|
||||||
|
|
||||||
|
public abstract byte getDataStructureType();
|
||||||
|
|
||||||
|
public abstract DataStructure createObject();
|
||||||
|
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
public int tightMarshalLong1(OpenWireFormat wireFormat, long o, BooleanStream bs) throws IOException {
|
||||||
|
if (o == 0) {
|
||||||
|
bs.writeBoolean(false);
|
||||||
|
bs.writeBoolean(false);
|
||||||
|
return 0;
|
||||||
|
} else if ((o & 0xFFFFFFFFFFFF0000L) == 0) {
|
||||||
|
bs.writeBoolean(false);
|
||||||
|
bs.writeBoolean(true);
|
||||||
|
return 2;
|
||||||
|
} else if ((o & 0xFFFFFFFF00000000L) == 0) {
|
||||||
|
bs.writeBoolean(true);
|
||||||
|
bs.writeBoolean(false);
|
||||||
|
return 4;
|
||||||
|
} else {
|
||||||
|
bs.writeBoolean(true);
|
||||||
|
bs.writeBoolean(true);
|
||||||
|
return 8;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void tightMarshalLong2(OpenWireFormat wireFormat, long o, DataOutput dataOut, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
dataOut.writeLong(o);
|
||||||
|
} else {
|
||||||
|
dataOut.writeInt((int)o);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
dataOut.writeShort((int)o);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public long tightUnmarshalLong(OpenWireFormat wireFormat, DataInput dataIn, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
return dataIn.readLong();
|
||||||
|
} else {
|
||||||
|
return toLong(dataIn.readInt());
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
return toLong(dataIn.readShort());
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected long toLong(short value) {
|
||||||
|
// lets handle negative values
|
||||||
|
long answer = value;
|
||||||
|
return answer & 0xffffL;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected long toLong(int value) {
|
||||||
|
// lets handle negative values
|
||||||
|
long answer = value;
|
||||||
|
return answer & 0xffffffffL;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected DataStructure tightUnmarsalNestedObject(OpenWireFormat wireFormat, DataInput dataIn,
|
||||||
|
BooleanStream bs) throws IOException {
|
||||||
|
return wireFormat.tightUnmarshalNestedObject(dataIn, bs);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int tightMarshalNestedObject1(OpenWireFormat wireFormat, DataStructure o, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
return wireFormat.tightMarshalNestedObject1(o, bs);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void tightMarshalNestedObject2(OpenWireFormat wireFormat, DataStructure o, DataOutput dataOut,
|
||||||
|
BooleanStream bs) throws IOException {
|
||||||
|
wireFormat.tightMarshalNestedObject2(o, dataOut, bs);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected DataStructure tightUnmarsalCachedObject(OpenWireFormat wireFormat, DataInput dataIn,
|
||||||
|
BooleanStream bs) throws IOException {
|
||||||
|
if (wireFormat.isCacheEnabled()) {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short index = dataIn.readShort();
|
||||||
|
DataStructure object = wireFormat.tightUnmarshalNestedObject(dataIn, bs);
|
||||||
|
wireFormat.setInUnmarshallCache(index, object);
|
||||||
|
return object;
|
||||||
|
} else {
|
||||||
|
short index = dataIn.readShort();
|
||||||
|
return wireFormat.getFromUnmarshallCache(index);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
return wireFormat.tightUnmarshalNestedObject(dataIn, bs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int tightMarshalCachedObject1(OpenWireFormat wireFormat, DataStructure o, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
if (wireFormat.isCacheEnabled()) {
|
||||||
|
Short index = wireFormat.getMarshallCacheIndex(o);
|
||||||
|
bs.writeBoolean(index == null);
|
||||||
|
if (index == null) {
|
||||||
|
int rc = wireFormat.tightMarshalNestedObject1(o, bs);
|
||||||
|
wireFormat.addToMarshallCache(o);
|
||||||
|
return 2 + rc;
|
||||||
|
} else {
|
||||||
|
return 2;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
return wireFormat.tightMarshalNestedObject1(o, bs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void tightMarshalCachedObject2(OpenWireFormat wireFormat, DataStructure o, DataOutput dataOut,
|
||||||
|
BooleanStream bs) throws IOException {
|
||||||
|
if (wireFormat.isCacheEnabled()) {
|
||||||
|
Short index = wireFormat.getMarshallCacheIndex(o);
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
dataOut.writeShort(index.shortValue());
|
||||||
|
wireFormat.tightMarshalNestedObject2(o, dataOut, bs);
|
||||||
|
} else {
|
||||||
|
dataOut.writeShort(index.shortValue());
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
wireFormat.tightMarshalNestedObject2(o, dataOut, bs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Throwable tightUnmarsalThrowable(OpenWireFormat wireFormat, DataInput dataIn, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
String clazz = tightUnmarshalString(dataIn, bs);
|
||||||
|
String message = tightUnmarshalString(dataIn, bs);
|
||||||
|
Throwable o = createThrowable(clazz, message);
|
||||||
|
if (wireFormat.isStackTraceEnabled()) {
|
||||||
|
if (STACK_TRACE_ELEMENT_CONSTRUCTOR != null) {
|
||||||
|
StackTraceElement ss[] = new StackTraceElement[dataIn.readShort()];
|
||||||
|
for (int i = 0; i < ss.length; i++) {
|
||||||
|
try {
|
||||||
|
ss[i] = (StackTraceElement)STACK_TRACE_ELEMENT_CONSTRUCTOR
|
||||||
|
.newInstance(new Object[] {tightUnmarshalString(dataIn, bs),
|
||||||
|
tightUnmarshalString(dataIn, bs),
|
||||||
|
tightUnmarshalString(dataIn, bs),
|
||||||
|
Integer.valueOf(dataIn.readInt())});
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw e;
|
||||||
|
} catch (Throwable e) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
o.setStackTrace(ss);
|
||||||
|
} else {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
for (int i = 0; i < size; i++) {
|
||||||
|
tightUnmarshalString(dataIn, bs);
|
||||||
|
tightUnmarshalString(dataIn, bs);
|
||||||
|
tightUnmarshalString(dataIn, bs);
|
||||||
|
dataIn.readInt();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
o.initCause(tightUnmarsalThrowable(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
return o;
|
||||||
|
} else {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Throwable createThrowable(String className, String message) {
|
||||||
|
try {
|
||||||
|
Class clazz = Class.forName(className, false, BaseDataStreamMarshaller.class.getClassLoader());
|
||||||
|
Constructor constructor = clazz.getConstructor(new Class[] {String.class});
|
||||||
|
return (Throwable)constructor.newInstance(new Object[] {message});
|
||||||
|
} catch (Throwable e) {
|
||||||
|
return new Throwable(className + ": " + message);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int tightMarshalThrowable1(OpenWireFormat wireFormat, Throwable o, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
if (o == null) {
|
||||||
|
bs.writeBoolean(false);
|
||||||
|
return 0;
|
||||||
|
} else {
|
||||||
|
int rc = 0;
|
||||||
|
bs.writeBoolean(true);
|
||||||
|
rc += tightMarshalString1(o.getClass().getName(), bs);
|
||||||
|
rc += tightMarshalString1(o.getMessage(), bs);
|
||||||
|
if (wireFormat.isStackTraceEnabled()) {
|
||||||
|
rc += 2;
|
||||||
|
StackTraceElement[] stackTrace = o.getStackTrace();
|
||||||
|
for (int i = 0; i < stackTrace.length; i++) {
|
||||||
|
StackTraceElement element = stackTrace[i];
|
||||||
|
rc += tightMarshalString1(element.getClassName(), bs);
|
||||||
|
rc += tightMarshalString1(element.getMethodName(), bs);
|
||||||
|
rc += tightMarshalString1(element.getFileName(), bs);
|
||||||
|
rc += 4;
|
||||||
|
}
|
||||||
|
rc += tightMarshalThrowable1(wireFormat, o.getCause(), bs);
|
||||||
|
}
|
||||||
|
return rc;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void tightMarshalThrowable2(OpenWireFormat wireFormat, Throwable o, DataOutput dataOut,
|
||||||
|
BooleanStream bs) throws IOException {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
tightMarshalString2(o.getClass().getName(), dataOut, bs);
|
||||||
|
tightMarshalString2(o.getMessage(), dataOut, bs);
|
||||||
|
if (wireFormat.isStackTraceEnabled()) {
|
||||||
|
StackTraceElement[] stackTrace = o.getStackTrace();
|
||||||
|
dataOut.writeShort(stackTrace.length);
|
||||||
|
for (int i = 0; i < stackTrace.length; i++) {
|
||||||
|
StackTraceElement element = stackTrace[i];
|
||||||
|
tightMarshalString2(element.getClassName(), dataOut, bs);
|
||||||
|
tightMarshalString2(element.getMethodName(), dataOut, bs);
|
||||||
|
tightMarshalString2(element.getFileName(), dataOut, bs);
|
||||||
|
dataOut.writeInt(element.getLineNumber());
|
||||||
|
}
|
||||||
|
tightMarshalThrowable2(wireFormat, o.getCause(), dataOut, bs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
protected String tightUnmarshalString(DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
int size = dataIn.readShort();
|
||||||
|
byte data[] = new byte[size];
|
||||||
|
dataIn.readFully(data);
|
||||||
|
// Yes deprecated, but we know what we are doing.
|
||||||
|
// This allows us to create a String from a ASCII byte array. (no UTF-8 decoding)
|
||||||
|
return new String(data, 0);
|
||||||
|
} else {
|
||||||
|
return dataIn.readUTF();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int tightMarshalString1(String value, BooleanStream bs) throws IOException {
|
||||||
|
bs.writeBoolean(value != null);
|
||||||
|
if (value != null) {
|
||||||
|
|
||||||
|
int strlen = value.length();
|
||||||
|
int utflen = 0;
|
||||||
|
char[] charr = new char[strlen];
|
||||||
|
int c = 0;
|
||||||
|
boolean isOnlyAscii = true;
|
||||||
|
|
||||||
|
value.getChars(0, strlen, charr, 0);
|
||||||
|
|
||||||
|
for (int i = 0; i < strlen; i++) {
|
||||||
|
c = charr[i];
|
||||||
|
if ((c >= 0x0001) && (c <= 0x007F)) {
|
||||||
|
utflen++;
|
||||||
|
} else if (c > 0x07FF) {
|
||||||
|
utflen += 3;
|
||||||
|
isOnlyAscii = false;
|
||||||
|
} else {
|
||||||
|
isOnlyAscii = false;
|
||||||
|
utflen += 2;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (utflen >= Short.MAX_VALUE) {
|
||||||
|
throw new IOException("Encountered a String value that is too long to encode.");
|
||||||
|
}
|
||||||
|
bs.writeBoolean(isOnlyAscii);
|
||||||
|
return utflen + 2;
|
||||||
|
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void tightMarshalString2(String value, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
// If we verified it only holds ascii values
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
dataOut.writeShort(value.length());
|
||||||
|
dataOut.writeBytes(value);
|
||||||
|
} else {
|
||||||
|
dataOut.writeUTF(value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int tightMarshalObjectArray1(OpenWireFormat wireFormat, DataStructure[] objects,
|
||||||
|
BooleanStream bs) throws IOException {
|
||||||
|
if (objects != null) {
|
||||||
|
int rc = 0;
|
||||||
|
bs.writeBoolean(true);
|
||||||
|
rc += 2;
|
||||||
|
for (int i = 0; i < objects.length; i++) {
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, objects[i], bs);
|
||||||
|
}
|
||||||
|
return rc;
|
||||||
|
} else {
|
||||||
|
bs.writeBoolean(false);
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void tightMarshalObjectArray2(OpenWireFormat wireFormat, DataStructure[] objects,
|
||||||
|
DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
dataOut.writeShort(objects.length);
|
||||||
|
for (int i = 0; i < objects.length; i++) {
|
||||||
|
tightMarshalNestedObject2(wireFormat, objects[i], dataOut, bs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int tightMarshalConstByteArray1(byte[] data, BooleanStream bs, int i) throws IOException {
|
||||||
|
return i;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void tightMarshalConstByteArray2(byte[] data, DataOutput dataOut, BooleanStream bs, int i)
|
||||||
|
throws IOException {
|
||||||
|
dataOut.write(data, 0, i);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected byte[] tightUnmarshalConstByteArray(DataInput dataIn, BooleanStream bs, int i)
|
||||||
|
throws IOException {
|
||||||
|
byte data[] = new byte[i];
|
||||||
|
dataIn.readFully(data);
|
||||||
|
return data;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int tightMarshalByteArray1(byte[] data, BooleanStream bs) throws IOException {
|
||||||
|
bs.writeBoolean(data != null);
|
||||||
|
if (data != null) {
|
||||||
|
return data.length + 4;
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void tightMarshalByteArray2(byte[] data, DataOutput dataOut, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
dataOut.writeInt(data.length);
|
||||||
|
dataOut.write(data);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected byte[] tightUnmarshalByteArray(DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
byte rc[] = null;
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
int size = dataIn.readInt();
|
||||||
|
rc = new byte[size];
|
||||||
|
dataIn.readFully(rc);
|
||||||
|
}
|
||||||
|
return rc;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int tightMarshalByteSequence1(ByteSequence data, BooleanStream bs) throws IOException {
|
||||||
|
bs.writeBoolean(data != null);
|
||||||
|
if (data != null) {
|
||||||
|
return data.getLength() + 4;
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void tightMarshalByteSequence2(ByteSequence data, DataOutput dataOut, BooleanStream bs)
|
||||||
|
throws IOException {
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
dataOut.writeInt(data.getLength());
|
||||||
|
dataOut.write(data.getData(), data.getOffset(), data.getLength());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ByteSequence tightUnmarshalByteSequence(DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
ByteSequence rc = null;
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
int size = dataIn.readInt();
|
||||||
|
byte[] t = new byte[size];
|
||||||
|
dataIn.readFully(t);
|
||||||
|
return new ByteSequence(t, 0, size);
|
||||||
|
}
|
||||||
|
return rc;
|
||||||
|
}
|
||||||
|
|
||||||
|
//
|
||||||
|
// The loose marshaling logic
|
||||||
|
//
|
||||||
|
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
public void looseMarshalLong(OpenWireFormat wireFormat, long o, DataOutput dataOut) throws IOException {
|
||||||
|
dataOut.writeLong(o);
|
||||||
|
}
|
||||||
|
|
||||||
|
public long looseUnmarshalLong(OpenWireFormat wireFormat, DataInput dataIn) throws IOException {
|
||||||
|
return dataIn.readLong();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected DataStructure looseUnmarsalNestedObject(OpenWireFormat wireFormat, DataInput dataIn)
|
||||||
|
throws IOException {
|
||||||
|
return wireFormat.looseUnmarshalNestedObject(dataIn);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void looseMarshalNestedObject(OpenWireFormat wireFormat, DataStructure o, DataOutput dataOut)
|
||||||
|
throws IOException {
|
||||||
|
wireFormat.looseMarshalNestedObject(o, dataOut);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected DataStructure looseUnmarsalCachedObject(OpenWireFormat wireFormat, DataInput dataIn)
|
||||||
|
throws IOException {
|
||||||
|
if (wireFormat.isCacheEnabled()) {
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short index = dataIn.readShort();
|
||||||
|
DataStructure object = wireFormat.looseUnmarshalNestedObject(dataIn);
|
||||||
|
wireFormat.setInUnmarshallCache(index, object);
|
||||||
|
return object;
|
||||||
|
} else {
|
||||||
|
short index = dataIn.readShort();
|
||||||
|
return wireFormat.getFromUnmarshallCache(index);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
return wireFormat.looseUnmarshalNestedObject(dataIn);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void looseMarshalCachedObject(OpenWireFormat wireFormat, DataStructure o, DataOutput dataOut)
|
||||||
|
throws IOException {
|
||||||
|
if (wireFormat.isCacheEnabled()) {
|
||||||
|
Short index = wireFormat.getMarshallCacheIndex(o);
|
||||||
|
dataOut.writeBoolean(index == null);
|
||||||
|
if (index == null) {
|
||||||
|
index = wireFormat.addToMarshallCache(o);
|
||||||
|
dataOut.writeShort(index.shortValue());
|
||||||
|
wireFormat.looseMarshalNestedObject(o, dataOut);
|
||||||
|
} else {
|
||||||
|
dataOut.writeShort(index.shortValue());
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
wireFormat.looseMarshalNestedObject(o, dataOut);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Throwable looseUnmarsalThrowable(OpenWireFormat wireFormat, DataInput dataIn)
|
||||||
|
throws IOException {
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
String clazz = looseUnmarshalString(dataIn);
|
||||||
|
String message = looseUnmarshalString(dataIn);
|
||||||
|
Throwable o = createThrowable(clazz, message);
|
||||||
|
if (wireFormat.isStackTraceEnabled()) {
|
||||||
|
if (STACK_TRACE_ELEMENT_CONSTRUCTOR != null) {
|
||||||
|
StackTraceElement ss[] = new StackTraceElement[dataIn.readShort()];
|
||||||
|
for (int i = 0; i < ss.length; i++) {
|
||||||
|
try {
|
||||||
|
ss[i] = (StackTraceElement)STACK_TRACE_ELEMENT_CONSTRUCTOR
|
||||||
|
.newInstance(new Object[] {looseUnmarshalString(dataIn),
|
||||||
|
looseUnmarshalString(dataIn),
|
||||||
|
looseUnmarshalString(dataIn),
|
||||||
|
Integer.valueOf(dataIn.readInt())});
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw e;
|
||||||
|
} catch (Throwable e) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
o.setStackTrace(ss);
|
||||||
|
} else {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
for (int i = 0; i < size; i++) {
|
||||||
|
looseUnmarshalString(dataIn);
|
||||||
|
looseUnmarshalString(dataIn);
|
||||||
|
looseUnmarshalString(dataIn);
|
||||||
|
dataIn.readInt();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
o.initCause(looseUnmarsalThrowable(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
return o;
|
||||||
|
} else {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void looseMarshalThrowable(OpenWireFormat wireFormat, Throwable o, DataOutput dataOut)
|
||||||
|
throws IOException {
|
||||||
|
dataOut.writeBoolean(o != null);
|
||||||
|
if (o != null) {
|
||||||
|
looseMarshalString(o.getClass().getName(), dataOut);
|
||||||
|
looseMarshalString(o.getMessage(), dataOut);
|
||||||
|
if (wireFormat.isStackTraceEnabled()) {
|
||||||
|
StackTraceElement[] stackTrace = o.getStackTrace();
|
||||||
|
dataOut.writeShort(stackTrace.length);
|
||||||
|
for (int i = 0; i < stackTrace.length; i++) {
|
||||||
|
StackTraceElement element = stackTrace[i];
|
||||||
|
looseMarshalString(element.getClassName(), dataOut);
|
||||||
|
looseMarshalString(element.getMethodName(), dataOut);
|
||||||
|
looseMarshalString(element.getFileName(), dataOut);
|
||||||
|
dataOut.writeInt(element.getLineNumber());
|
||||||
|
}
|
||||||
|
looseMarshalThrowable(wireFormat, o.getCause(), dataOut);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected String looseUnmarshalString(DataInput dataIn) throws IOException {
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
return dataIn.readUTF();
|
||||||
|
} else {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void looseMarshalString(String value, DataOutput dataOut) throws IOException {
|
||||||
|
dataOut.writeBoolean(value != null);
|
||||||
|
if (value != null) {
|
||||||
|
dataOut.writeUTF(value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void looseMarshalObjectArray(OpenWireFormat wireFormat, DataStructure[] objects,
|
||||||
|
DataOutput dataOut) throws IOException {
|
||||||
|
dataOut.writeBoolean(objects != null);
|
||||||
|
if (objects != null) {
|
||||||
|
dataOut.writeShort(objects.length);
|
||||||
|
for (int i = 0; i < objects.length; i++) {
|
||||||
|
looseMarshalNestedObject(wireFormat, objects[i], dataOut);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void looseMarshalConstByteArray(OpenWireFormat wireFormat, byte[] data, DataOutput dataOut,
|
||||||
|
int i) throws IOException {
|
||||||
|
dataOut.write(data, 0, i);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected byte[] looseUnmarshalConstByteArray(DataInput dataIn, int i) throws IOException {
|
||||||
|
byte data[] = new byte[i];
|
||||||
|
dataIn.readFully(data);
|
||||||
|
return data;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void looseMarshalByteArray(OpenWireFormat wireFormat, byte[] data, DataOutput dataOut)
|
||||||
|
throws IOException {
|
||||||
|
dataOut.writeBoolean(data != null);
|
||||||
|
if (data != null) {
|
||||||
|
dataOut.writeInt(data.length);
|
||||||
|
dataOut.write(data);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected byte[] looseUnmarshalByteArray(DataInput dataIn) throws IOException {
|
||||||
|
byte rc[] = null;
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
int size = dataIn.readInt();
|
||||||
|
rc = new byte[size];
|
||||||
|
dataIn.readFully(rc);
|
||||||
|
}
|
||||||
|
return rc;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void looseMarshalByteSequence(OpenWireFormat wireFormat, ByteSequence data, DataOutput dataOut)
|
||||||
|
throws IOException {
|
||||||
|
dataOut.writeBoolean(data != null);
|
||||||
|
if (data != null) {
|
||||||
|
dataOut.writeInt(data.getLength());
|
||||||
|
dataOut.write(data.getData(), data.getOffset(), data.getLength());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ByteSequence looseUnmarshalByteSequence(DataInput dataIn) throws IOException {
|
||||||
|
ByteSequence rc = null;
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
int size = dataIn.readInt();
|
||||||
|
byte[] t = new byte[size];
|
||||||
|
dataIn.readFully(t);
|
||||||
|
rc = new ByteSequence(t, 0, size);
|
||||||
|
}
|
||||||
|
return rc;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for BrokerIdMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class BrokerIdMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return BrokerId.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new BrokerId();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
BrokerId info = (BrokerId)o;
|
||||||
|
info.setValue(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
BrokerId info = (BrokerId)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getValue(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
BrokerId info = (BrokerId)o;
|
||||||
|
tightMarshalString2(info.getValue(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
BrokerId info = (BrokerId)o;
|
||||||
|
info.setValue(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
BrokerId info = (BrokerId)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getValue(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,206 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for BrokerInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class BrokerInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return BrokerInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new BrokerInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
BrokerInfo info = (BrokerInfo)o;
|
||||||
|
info.setBrokerId((org.apache.activemq.command.BrokerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setBrokerURL(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerInfo value[] = new org.apache.activemq.command.BrokerInfo[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerInfo) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setPeerBrokerInfos(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setPeerBrokerInfos(null);
|
||||||
|
}
|
||||||
|
info.setBrokerName(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setSlaveBroker(bs.readBoolean());
|
||||||
|
info.setMasterBroker(bs.readBoolean());
|
||||||
|
info.setFaultTolerantConfiguration(bs.readBoolean());
|
||||||
|
info.setDuplexConnection(bs.readBoolean());
|
||||||
|
info.setNetworkConnection(bs.readBoolean());
|
||||||
|
info.setConnectionId(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setBrokerUploadUrl(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setNetworkProperties(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
BrokerInfo info = (BrokerInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getBrokerId(), bs);
|
||||||
|
rc += tightMarshalString1(info.getBrokerURL(), bs);
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getPeerBrokerInfos(), bs);
|
||||||
|
rc += tightMarshalString1(info.getBrokerName(), bs);
|
||||||
|
bs.writeBoolean(info.isSlaveBroker());
|
||||||
|
bs.writeBoolean(info.isMasterBroker());
|
||||||
|
bs.writeBoolean(info.isFaultTolerantConfiguration());
|
||||||
|
bs.writeBoolean(info.isDuplexConnection());
|
||||||
|
bs.writeBoolean(info.isNetworkConnection());
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getConnectionId(), bs);
|
||||||
|
rc += tightMarshalString1(info.getBrokerUploadUrl(), bs);
|
||||||
|
rc += tightMarshalString1(info.getNetworkProperties(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
BrokerInfo info = (BrokerInfo)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getBrokerId(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getBrokerURL(), dataOut, bs);
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getPeerBrokerInfos(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getBrokerName(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalLong2(wireFormat, info.getConnectionId(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getBrokerUploadUrl(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getNetworkProperties(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
BrokerInfo info = (BrokerInfo)o;
|
||||||
|
info.setBrokerId((org.apache.activemq.command.BrokerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setBrokerURL(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerInfo value[] = new org.apache.activemq.command.BrokerInfo[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerInfo) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setPeerBrokerInfos(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setPeerBrokerInfos(null);
|
||||||
|
}
|
||||||
|
info.setBrokerName(looseUnmarshalString(dataIn));
|
||||||
|
info.setSlaveBroker(dataIn.readBoolean());
|
||||||
|
info.setMasterBroker(dataIn.readBoolean());
|
||||||
|
info.setFaultTolerantConfiguration(dataIn.readBoolean());
|
||||||
|
info.setDuplexConnection(dataIn.readBoolean());
|
||||||
|
info.setNetworkConnection(dataIn.readBoolean());
|
||||||
|
info.setConnectionId(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setBrokerUploadUrl(looseUnmarshalString(dataIn));
|
||||||
|
info.setNetworkProperties(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
BrokerInfo info = (BrokerInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getBrokerId(), dataOut);
|
||||||
|
looseMarshalString(info.getBrokerURL(), dataOut);
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getPeerBrokerInfos(), dataOut);
|
||||||
|
looseMarshalString(info.getBrokerName(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isSlaveBroker());
|
||||||
|
dataOut.writeBoolean(info.isMasterBroker());
|
||||||
|
dataOut.writeBoolean(info.isFaultTolerantConfiguration());
|
||||||
|
dataOut.writeBoolean(info.isDuplexConnection());
|
||||||
|
dataOut.writeBoolean(info.isNetworkConnection());
|
||||||
|
looseMarshalLong(wireFormat, info.getConnectionId(), dataOut);
|
||||||
|
looseMarshalString(info.getBrokerUploadUrl(), dataOut);
|
||||||
|
looseMarshalString(info.getNetworkProperties(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,161 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for BrokerSubscriptionInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class BrokerSubscriptionInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return BrokerSubscriptionInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new BrokerSubscriptionInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
BrokerSubscriptionInfo info = (BrokerSubscriptionInfo)o;
|
||||||
|
info.setBrokerId((org.apache.activemq.command.BrokerId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setBrokerName(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.ConsumerInfo value[] = new org.apache.activemq.command.ConsumerInfo[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.ConsumerInfo) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setSubscriptionInfos(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setSubscriptionInfos(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
BrokerSubscriptionInfo info = (BrokerSubscriptionInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getBrokerId(), bs);
|
||||||
|
rc += tightMarshalString1(info.getBrokerName(), bs);
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getSubscriptionInfos(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
BrokerSubscriptionInfo info = (BrokerSubscriptionInfo)o;
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getBrokerId(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getBrokerName(), dataOut, bs);
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getSubscriptionInfos(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
BrokerSubscriptionInfo info = (BrokerSubscriptionInfo)o;
|
||||||
|
info.setBrokerId((org.apache.activemq.command.BrokerId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setBrokerName(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.ConsumerInfo value[] = new org.apache.activemq.command.ConsumerInfo[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.ConsumerInfo) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setSubscriptionInfos(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setSubscriptionInfos(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
BrokerSubscriptionInfo info = (BrokerSubscriptionInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getBrokerId(), dataOut);
|
||||||
|
looseMarshalString(info.getBrokerName(), dataOut);
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getSubscriptionInfos(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,169 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ConnectionControlMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ConnectionControlMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ConnectionControl.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ConnectionControl();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ConnectionControl info = (ConnectionControl)o;
|
||||||
|
info.setClose(bs.readBoolean());
|
||||||
|
info.setExit(bs.readBoolean());
|
||||||
|
info.setFaultTolerant(bs.readBoolean());
|
||||||
|
info.setResume(bs.readBoolean());
|
||||||
|
info.setSuspend(bs.readBoolean());
|
||||||
|
info.setConnectedBrokers(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setReconnectTo(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setRebalanceConnection(bs.readBoolean());
|
||||||
|
info.setToken(tightUnmarshalByteArray(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ConnectionControl info = (ConnectionControl)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
bs.writeBoolean(info.isClose());
|
||||||
|
bs.writeBoolean(info.isExit());
|
||||||
|
bs.writeBoolean(info.isFaultTolerant());
|
||||||
|
bs.writeBoolean(info.isResume());
|
||||||
|
bs.writeBoolean(info.isSuspend());
|
||||||
|
rc += tightMarshalString1(info.getConnectedBrokers(), bs);
|
||||||
|
rc += tightMarshalString1(info.getReconnectTo(), bs);
|
||||||
|
bs.writeBoolean(info.isRebalanceConnection());
|
||||||
|
rc += tightMarshalByteArray1(info.getToken(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ConnectionControl info = (ConnectionControl)o;
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalString2(info.getConnectedBrokers(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getReconnectTo(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalByteArray2(info.getToken(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ConnectionControl info = (ConnectionControl)o;
|
||||||
|
info.setClose(dataIn.readBoolean());
|
||||||
|
info.setExit(dataIn.readBoolean());
|
||||||
|
info.setFaultTolerant(dataIn.readBoolean());
|
||||||
|
info.setResume(dataIn.readBoolean());
|
||||||
|
info.setSuspend(dataIn.readBoolean());
|
||||||
|
info.setConnectedBrokers(looseUnmarshalString(dataIn));
|
||||||
|
info.setReconnectTo(looseUnmarshalString(dataIn));
|
||||||
|
info.setRebalanceConnection(dataIn.readBoolean());
|
||||||
|
info.setToken(looseUnmarshalByteArray(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ConnectionControl info = (ConnectionControl)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
dataOut.writeBoolean(info.isClose());
|
||||||
|
dataOut.writeBoolean(info.isExit());
|
||||||
|
dataOut.writeBoolean(info.isFaultTolerant());
|
||||||
|
dataOut.writeBoolean(info.isResume());
|
||||||
|
dataOut.writeBoolean(info.isSuspend());
|
||||||
|
looseMarshalString(info.getConnectedBrokers(), dataOut);
|
||||||
|
looseMarshalString(info.getReconnectTo(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isRebalanceConnection());
|
||||||
|
looseMarshalByteArray(wireFormat, info.getToken(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,134 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ConnectionErrorMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ConnectionErrorMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ConnectionError.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ConnectionError();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ConnectionError info = (ConnectionError)o;
|
||||||
|
info.setException((java.lang.Throwable) tightUnmarsalThrowable(wireFormat, dataIn, bs));
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ConnectionError info = (ConnectionError)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalThrowable1(wireFormat, info.getException(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getConnectionId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ConnectionError info = (ConnectionError)o;
|
||||||
|
tightMarshalThrowable2(wireFormat, info.getException(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getConnectionId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ConnectionError info = (ConnectionError)o;
|
||||||
|
info.setException((java.lang.Throwable) looseUnmarsalThrowable(wireFormat, dataIn));
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ConnectionError info = (ConnectionError)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalThrowable(wireFormat, info.getException(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getConnectionId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ConnectionIdMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ConnectionIdMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ConnectionId.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ConnectionId();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ConnectionId info = (ConnectionId)o;
|
||||||
|
info.setValue(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ConnectionId info = (ConnectionId)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getValue(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ConnectionId info = (ConnectionId)o;
|
||||||
|
tightMarshalString2(info.getValue(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ConnectionId info = (ConnectionId)o;
|
||||||
|
info.setValue(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ConnectionId info = (ConnectionId)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getValue(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,201 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ConnectionInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ConnectionInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ConnectionInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ConnectionInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ConnectionInfo info = (ConnectionInfo)o;
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setClientId(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setPassword(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setUserName(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
info.setBrokerMasterConnector(bs.readBoolean());
|
||||||
|
info.setManageable(bs.readBoolean());
|
||||||
|
info.setClientMaster(bs.readBoolean());
|
||||||
|
info.setFaultTolerant(bs.readBoolean());
|
||||||
|
info.setFailoverReconnect(bs.readBoolean());
|
||||||
|
info.setClientIp(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ConnectionInfo info = (ConnectionInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConnectionId(), bs);
|
||||||
|
rc += tightMarshalString1(info.getClientId(), bs);
|
||||||
|
rc += tightMarshalString1(info.getPassword(), bs);
|
||||||
|
rc += tightMarshalString1(info.getUserName(), bs);
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getBrokerPath(), bs);
|
||||||
|
bs.writeBoolean(info.isBrokerMasterConnector());
|
||||||
|
bs.writeBoolean(info.isManageable());
|
||||||
|
bs.writeBoolean(info.isClientMaster());
|
||||||
|
bs.writeBoolean(info.isFaultTolerant());
|
||||||
|
bs.writeBoolean(info.isFailoverReconnect());
|
||||||
|
rc += tightMarshalString1(info.getClientIp(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ConnectionInfo info = (ConnectionInfo)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConnectionId(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getClientId(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getPassword(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getUserName(), dataOut, bs);
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getBrokerPath(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalString2(info.getClientIp(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ConnectionInfo info = (ConnectionInfo)o;
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setClientId(looseUnmarshalString(dataIn));
|
||||||
|
info.setPassword(looseUnmarshalString(dataIn));
|
||||||
|
info.setUserName(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
info.setBrokerMasterConnector(dataIn.readBoolean());
|
||||||
|
info.setManageable(dataIn.readBoolean());
|
||||||
|
info.setClientMaster(dataIn.readBoolean());
|
||||||
|
info.setFaultTolerant(dataIn.readBoolean());
|
||||||
|
info.setFailoverReconnect(dataIn.readBoolean());
|
||||||
|
info.setClientIp(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ConnectionInfo info = (ConnectionInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConnectionId(), dataOut);
|
||||||
|
looseMarshalString(info.getClientId(), dataOut);
|
||||||
|
looseMarshalString(info.getPassword(), dataOut);
|
||||||
|
looseMarshalString(info.getUserName(), dataOut);
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getBrokerPath(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isBrokerMasterConnector());
|
||||||
|
dataOut.writeBoolean(info.isManageable());
|
||||||
|
dataOut.writeBoolean(info.isClientMaster());
|
||||||
|
dataOut.writeBoolean(info.isFaultTolerant());
|
||||||
|
dataOut.writeBoolean(info.isFailoverReconnect());
|
||||||
|
looseMarshalString(info.getClientIp(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,158 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ConsumerControlMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ConsumerControlMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ConsumerControl.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ConsumerControl();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ConsumerControl info = (ConsumerControl)o;
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setClose(bs.readBoolean());
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setPrefetch(dataIn.readInt());
|
||||||
|
info.setFlush(bs.readBoolean());
|
||||||
|
info.setStart(bs.readBoolean());
|
||||||
|
info.setStop(bs.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ConsumerControl info = (ConsumerControl)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
bs.writeBoolean(info.isClose());
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getConsumerId(), bs);
|
||||||
|
bs.writeBoolean(info.isFlush());
|
||||||
|
bs.writeBoolean(info.isStart());
|
||||||
|
bs.writeBoolean(info.isStop());
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ConsumerControl info = (ConsumerControl)o;
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getConsumerId(), dataOut, bs);
|
||||||
|
dataOut.writeInt(info.getPrefetch());
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ConsumerControl info = (ConsumerControl)o;
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setClose(dataIn.readBoolean());
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setPrefetch(dataIn.readInt());
|
||||||
|
info.setFlush(dataIn.readBoolean());
|
||||||
|
info.setStart(dataIn.readBoolean());
|
||||||
|
info.setStop(dataIn.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ConsumerControl info = (ConsumerControl)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isClose());
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getConsumerId(), dataOut);
|
||||||
|
dataOut.writeInt(info.getPrefetch());
|
||||||
|
dataOut.writeBoolean(info.isFlush());
|
||||||
|
dataOut.writeBoolean(info.isStart());
|
||||||
|
dataOut.writeBoolean(info.isStop());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,139 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ConsumerIdMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ConsumerIdMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ConsumerId.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ConsumerId();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ConsumerId info = (ConsumerId)o;
|
||||||
|
info.setConnectionId(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setSessionId(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setValue(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ConsumerId info = (ConsumerId)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getConnectionId(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getSessionId(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getValue(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ConsumerId info = (ConsumerId)o;
|
||||||
|
tightMarshalString2(info.getConnectionId(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getSessionId(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getValue(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ConsumerId info = (ConsumerId)o;
|
||||||
|
info.setConnectionId(looseUnmarshalString(dataIn));
|
||||||
|
info.setSessionId(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setValue(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ConsumerId info = (ConsumerId)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getConnectionId(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getSessionId(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getValue(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,260 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ConsumerInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ConsumerInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ConsumerInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ConsumerInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ConsumerInfo info = (ConsumerInfo)o;
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setBrowser(bs.readBoolean());
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setPrefetchSize(dataIn.readInt());
|
||||||
|
info.setMaximumPendingMessageLimit(dataIn.readInt());
|
||||||
|
info.setDispatchAsync(bs.readBoolean());
|
||||||
|
info.setSelector(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setClientId(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setSubscriptionName(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setNoLocal(bs.readBoolean());
|
||||||
|
info.setExclusive(bs.readBoolean());
|
||||||
|
info.setRetroactive(bs.readBoolean());
|
||||||
|
info.setPriority(dataIn.readByte());
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
info.setAdditionalPredicate((org.apache.activemq.filter.BooleanExpression) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setNetworkSubscription(bs.readBoolean());
|
||||||
|
info.setOptimizedAcknowledge(bs.readBoolean());
|
||||||
|
info.setNoRangeAcks(bs.readBoolean());
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.ConsumerId value[] = new org.apache.activemq.command.ConsumerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.ConsumerId) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setNetworkConsumerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setNetworkConsumerPath(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ConsumerInfo info = (ConsumerInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConsumerId(), bs);
|
||||||
|
bs.writeBoolean(info.isBrowser());
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
bs.writeBoolean(info.isDispatchAsync());
|
||||||
|
rc += tightMarshalString1(info.getSelector(), bs);
|
||||||
|
rc += tightMarshalString1(info.getClientId(), bs);
|
||||||
|
rc += tightMarshalString1(info.getSubscriptionName(), bs);
|
||||||
|
bs.writeBoolean(info.isNoLocal());
|
||||||
|
bs.writeBoolean(info.isExclusive());
|
||||||
|
bs.writeBoolean(info.isRetroactive());
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getBrokerPath(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getAdditionalPredicate(), bs);
|
||||||
|
bs.writeBoolean(info.isNetworkSubscription());
|
||||||
|
bs.writeBoolean(info.isOptimizedAcknowledge());
|
||||||
|
bs.writeBoolean(info.isNoRangeAcks());
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getNetworkConsumerPath(), bs);
|
||||||
|
|
||||||
|
return rc + 9;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ConsumerInfo info = (ConsumerInfo)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConsumerId(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
dataOut.writeInt(info.getPrefetchSize());
|
||||||
|
dataOut.writeInt(info.getMaximumPendingMessageLimit());
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalString2(info.getSelector(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getClientId(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getSubscriptionName(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
dataOut.writeByte(info.getPriority());
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getBrokerPath(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getAdditionalPredicate(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getNetworkConsumerPath(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ConsumerInfo info = (ConsumerInfo)o;
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setBrowser(dataIn.readBoolean());
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setPrefetchSize(dataIn.readInt());
|
||||||
|
info.setMaximumPendingMessageLimit(dataIn.readInt());
|
||||||
|
info.setDispatchAsync(dataIn.readBoolean());
|
||||||
|
info.setSelector(looseUnmarshalString(dataIn));
|
||||||
|
info.setClientId(looseUnmarshalString(dataIn));
|
||||||
|
info.setSubscriptionName(looseUnmarshalString(dataIn));
|
||||||
|
info.setNoLocal(dataIn.readBoolean());
|
||||||
|
info.setExclusive(dataIn.readBoolean());
|
||||||
|
info.setRetroactive(dataIn.readBoolean());
|
||||||
|
info.setPriority(dataIn.readByte());
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
info.setAdditionalPredicate((org.apache.activemq.filter.BooleanExpression) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setNetworkSubscription(dataIn.readBoolean());
|
||||||
|
info.setOptimizedAcknowledge(dataIn.readBoolean());
|
||||||
|
info.setNoRangeAcks(dataIn.readBoolean());
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.ConsumerId value[] = new org.apache.activemq.command.ConsumerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.ConsumerId) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setNetworkConsumerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setNetworkConsumerPath(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ConsumerInfo info = (ConsumerInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConsumerId(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isBrowser());
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
dataOut.writeInt(info.getPrefetchSize());
|
||||||
|
dataOut.writeInt(info.getMaximumPendingMessageLimit());
|
||||||
|
dataOut.writeBoolean(info.isDispatchAsync());
|
||||||
|
looseMarshalString(info.getSelector(), dataOut);
|
||||||
|
looseMarshalString(info.getClientId(), dataOut);
|
||||||
|
looseMarshalString(info.getSubscriptionName(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isNoLocal());
|
||||||
|
dataOut.writeBoolean(info.isExclusive());
|
||||||
|
dataOut.writeBoolean(info.isRetroactive());
|
||||||
|
dataOut.writeByte(info.getPriority());
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getBrokerPath(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getAdditionalPredicate(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isNetworkSubscription());
|
||||||
|
dataOut.writeBoolean(info.isOptimizedAcknowledge());
|
||||||
|
dataOut.writeBoolean(info.isNoRangeAcks());
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getNetworkConsumerPath(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ControlCommandMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ControlCommandMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ControlCommand.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ControlCommand();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ControlCommand info = (ControlCommand)o;
|
||||||
|
info.setCommand(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ControlCommand info = (ControlCommand)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getCommand(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ControlCommand info = (ControlCommand)o;
|
||||||
|
tightMarshalString2(info.getCommand(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ControlCommand info = (ControlCommand)o;
|
||||||
|
info.setCommand(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ControlCommand info = (ControlCommand)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getCommand(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,151 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for DataArrayResponseMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class DataArrayResponseMarshaller extends ResponseMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return DataArrayResponse.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new DataArrayResponse();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
DataArrayResponse info = (DataArrayResponse)o;
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.DataStructure value[] = new org.apache.activemq.command.DataStructure[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.DataStructure) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setData(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setData(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
DataArrayResponse info = (DataArrayResponse)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getData(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
DataArrayResponse info = (DataArrayResponse)o;
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getData(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
DataArrayResponse info = (DataArrayResponse)o;
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.DataStructure value[] = new org.apache.activemq.command.DataStructure[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.DataStructure) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setData(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setData(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
DataArrayResponse info = (DataArrayResponse)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getData(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for DataResponseMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class DataResponseMarshaller extends ResponseMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return DataResponse.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new DataResponse();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
DataResponse info = (DataResponse)o;
|
||||||
|
info.setData((org.apache.activemq.command.DataStructure) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
DataResponse info = (DataResponse)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getData(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
DataResponse info = (DataResponse)o;
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getData(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
DataResponse info = (DataResponse)o;
|
||||||
|
info.setData((org.apache.activemq.command.DataStructure) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
DataResponse info = (DataResponse)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getData(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,170 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for DestinationInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class DestinationInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return DestinationInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new DestinationInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
DestinationInfo info = (DestinationInfo)o;
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setOperationType(dataIn.readByte());
|
||||||
|
info.setTimeout(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
DestinationInfo info = (DestinationInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConnectionId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getTimeout(), bs);
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getBrokerPath(), bs);
|
||||||
|
|
||||||
|
return rc + 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
DestinationInfo info = (DestinationInfo)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConnectionId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
dataOut.writeByte(info.getOperationType());
|
||||||
|
tightMarshalLong2(wireFormat, info.getTimeout(), dataOut, bs);
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getBrokerPath(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
DestinationInfo info = (DestinationInfo)o;
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setOperationType(dataIn.readByte());
|
||||||
|
info.setTimeout(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
DestinationInfo info = (DestinationInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConnectionId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
dataOut.writeByte(info.getOperationType());
|
||||||
|
looseMarshalLong(wireFormat, info.getTimeout(), dataOut);
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getBrokerPath(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,134 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for DiscoveryEventMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class DiscoveryEventMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return DiscoveryEvent.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new DiscoveryEvent();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
DiscoveryEvent info = (DiscoveryEvent)o;
|
||||||
|
info.setServiceName(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setBrokerName(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
DiscoveryEvent info = (DiscoveryEvent)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getServiceName(), bs);
|
||||||
|
rc += tightMarshalString1(info.getBrokerName(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
DiscoveryEvent info = (DiscoveryEvent)o;
|
||||||
|
tightMarshalString2(info.getServiceName(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getBrokerName(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
DiscoveryEvent info = (DiscoveryEvent)o;
|
||||||
|
info.setServiceName(looseUnmarshalString(dataIn));
|
||||||
|
info.setBrokerName(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
DiscoveryEvent info = (DiscoveryEvent)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getServiceName(), dataOut);
|
||||||
|
looseMarshalString(info.getBrokerName(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ExceptionResponseMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ExceptionResponseMarshaller extends ResponseMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ExceptionResponse.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ExceptionResponse();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ExceptionResponse info = (ExceptionResponse)o;
|
||||||
|
info.setException((java.lang.Throwable) tightUnmarsalThrowable(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ExceptionResponse info = (ExceptionResponse)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalThrowable1(wireFormat, info.getException(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ExceptionResponse info = (ExceptionResponse)o;
|
||||||
|
tightMarshalThrowable2(wireFormat, info.getException(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ExceptionResponse info = (ExceptionResponse)o;
|
||||||
|
info.setException((java.lang.Throwable) looseUnmarsalThrowable(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ExceptionResponse info = (ExceptionResponse)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalThrowable(wireFormat, info.getException(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for FlushCommandMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class FlushCommandMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return FlushCommand.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new FlushCommand();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,128 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for IntegerResponseMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class IntegerResponseMarshaller extends ResponseMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return IntegerResponse.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new IntegerResponse();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
IntegerResponse info = (IntegerResponse)o;
|
||||||
|
info.setResult(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
IntegerResponse info = (IntegerResponse)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
IntegerResponse info = (IntegerResponse)o;
|
||||||
|
dataOut.writeInt(info.getResult());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
IntegerResponse info = (IntegerResponse)o;
|
||||||
|
info.setResult(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
IntegerResponse info = (IntegerResponse)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
dataOut.writeInt(info.getResult());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,134 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for JournalQueueAckMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class JournalQueueAckMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return JournalQueueAck.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new JournalQueueAck();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
JournalQueueAck info = (JournalQueueAck)o;
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setMessageAck((org.apache.activemq.command.MessageAck) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
JournalQueueAck info = (JournalQueueAck)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getMessageAck(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
JournalQueueAck info = (JournalQueueAck)o;
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getMessageAck(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
JournalQueueAck info = (JournalQueueAck)o;
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setMessageAck((org.apache.activemq.command.MessageAck) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
JournalQueueAck info = (JournalQueueAck)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getMessageAck(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,154 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for JournalTopicAckMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class JournalTopicAckMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return JournalTopicAck.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new JournalTopicAck();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
JournalTopicAck info = (JournalTopicAck)o;
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setMessageId((org.apache.activemq.command.MessageId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setMessageSequenceId(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setSubscritionName(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setClientId(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
JournalTopicAck info = (JournalTopicAck)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getMessageId(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getMessageSequenceId(), bs);
|
||||||
|
rc += tightMarshalString1(info.getSubscritionName(), bs);
|
||||||
|
rc += tightMarshalString1(info.getClientId(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getTransactionId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
JournalTopicAck info = (JournalTopicAck)o;
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getMessageId(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getMessageSequenceId(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getSubscritionName(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getClientId(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getTransactionId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
JournalTopicAck info = (JournalTopicAck)o;
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setMessageId((org.apache.activemq.command.MessageId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setMessageSequenceId(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setSubscritionName(looseUnmarshalString(dataIn));
|
||||||
|
info.setClientId(looseUnmarshalString(dataIn));
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
JournalTopicAck info = (JournalTopicAck)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getMessageId(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getMessageSequenceId(), dataOut);
|
||||||
|
looseMarshalString(info.getSubscritionName(), dataOut);
|
||||||
|
looseMarshalString(info.getClientId(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getTransactionId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for JournalTraceMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class JournalTraceMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return JournalTrace.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new JournalTrace();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
JournalTrace info = (JournalTrace)o;
|
||||||
|
info.setMessage(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
JournalTrace info = (JournalTrace)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getMessage(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
JournalTrace info = (JournalTrace)o;
|
||||||
|
tightMarshalString2(info.getMessage(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
JournalTrace info = (JournalTrace)o;
|
||||||
|
info.setMessage(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
JournalTrace info = (JournalTrace)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getMessage(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,138 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for JournalTransactionMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class JournalTransactionMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return JournalTransaction.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new JournalTransaction();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
JournalTransaction info = (JournalTransaction)o;
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setType(dataIn.readByte());
|
||||||
|
info.setWasPrepared(bs.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
JournalTransaction info = (JournalTransaction)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getTransactionId(), bs);
|
||||||
|
bs.writeBoolean(info.getWasPrepared());
|
||||||
|
|
||||||
|
return rc + 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
JournalTransaction info = (JournalTransaction)o;
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getTransactionId(), dataOut, bs);
|
||||||
|
dataOut.writeByte(info.getType());
|
||||||
|
bs.readBoolean();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
JournalTransaction info = (JournalTransaction)o;
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setType(dataIn.readByte());
|
||||||
|
info.setWasPrepared(dataIn.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
JournalTransaction info = (JournalTransaction)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getTransactionId(), dataOut);
|
||||||
|
dataOut.writeByte(info.getType());
|
||||||
|
dataOut.writeBoolean(info.getWasPrepared());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for KeepAliveInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class KeepAliveInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return KeepAliveInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new KeepAliveInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for LastPartialCommandMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class LastPartialCommandMarshaller extends PartialCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return LastPartialCommand.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new LastPartialCommand();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,134 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for LocalTransactionIdMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class LocalTransactionIdMarshaller extends TransactionIdMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return LocalTransactionId.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new LocalTransactionId();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
LocalTransactionId info = (LocalTransactionId)o;
|
||||||
|
info.setValue(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
LocalTransactionId info = (LocalTransactionId)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getValue(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConnectionId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
LocalTransactionId info = (LocalTransactionId)o;
|
||||||
|
tightMarshalLong2(wireFormat, info.getValue(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConnectionId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
LocalTransactionId info = (LocalTransactionId)o;
|
||||||
|
info.setValue(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
LocalTransactionId info = (LocalTransactionId)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getValue(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConnectionId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,110 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.DataStreamMarshaller;
|
||||||
|
import org.apache.activemq.openwire.OpenWireFormat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* MarshallerFactory for Open Wire Format.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class MarshallerFactory {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a Map of command type -> Marshallers
|
||||||
|
*/
|
||||||
|
static final private DataStreamMarshaller marshaller[] = new DataStreamMarshaller[256];
|
||||||
|
static {
|
||||||
|
|
||||||
|
add(new ActiveMQBlobMessageMarshaller());
|
||||||
|
add(new ActiveMQBytesMessageMarshaller());
|
||||||
|
add(new ActiveMQMapMessageMarshaller());
|
||||||
|
add(new ActiveMQMessageMarshaller());
|
||||||
|
add(new ActiveMQObjectMessageMarshaller());
|
||||||
|
add(new ActiveMQQueueMarshaller());
|
||||||
|
add(new ActiveMQStreamMessageMarshaller());
|
||||||
|
add(new ActiveMQTempQueueMarshaller());
|
||||||
|
add(new ActiveMQTempTopicMarshaller());
|
||||||
|
add(new ActiveMQTextMessageMarshaller());
|
||||||
|
add(new ActiveMQTopicMarshaller());
|
||||||
|
add(new BrokerIdMarshaller());
|
||||||
|
add(new BrokerInfoMarshaller());
|
||||||
|
add(new BrokerSubscriptionInfoMarshaller());
|
||||||
|
add(new ConnectionControlMarshaller());
|
||||||
|
add(new ConnectionErrorMarshaller());
|
||||||
|
add(new ConnectionIdMarshaller());
|
||||||
|
add(new ConnectionInfoMarshaller());
|
||||||
|
add(new ConsumerControlMarshaller());
|
||||||
|
add(new ConsumerIdMarshaller());
|
||||||
|
add(new ConsumerInfoMarshaller());
|
||||||
|
add(new ControlCommandMarshaller());
|
||||||
|
add(new DataArrayResponseMarshaller());
|
||||||
|
add(new DataResponseMarshaller());
|
||||||
|
add(new DestinationInfoMarshaller());
|
||||||
|
add(new DiscoveryEventMarshaller());
|
||||||
|
add(new ExceptionResponseMarshaller());
|
||||||
|
add(new FlushCommandMarshaller());
|
||||||
|
add(new IntegerResponseMarshaller());
|
||||||
|
add(new JournalQueueAckMarshaller());
|
||||||
|
add(new JournalTopicAckMarshaller());
|
||||||
|
add(new JournalTraceMarshaller());
|
||||||
|
add(new JournalTransactionMarshaller());
|
||||||
|
add(new KeepAliveInfoMarshaller());
|
||||||
|
add(new LastPartialCommandMarshaller());
|
||||||
|
add(new LocalTransactionIdMarshaller());
|
||||||
|
add(new MessageAckMarshaller());
|
||||||
|
add(new MessageDispatchMarshaller());
|
||||||
|
add(new MessageDispatchNotificationMarshaller());
|
||||||
|
add(new MessageIdMarshaller());
|
||||||
|
add(new MessagePullMarshaller());
|
||||||
|
add(new NetworkBridgeFilterMarshaller());
|
||||||
|
add(new PartialCommandMarshaller());
|
||||||
|
add(new ProducerAckMarshaller());
|
||||||
|
add(new ProducerIdMarshaller());
|
||||||
|
add(new ProducerInfoMarshaller());
|
||||||
|
add(new RemoveInfoMarshaller());
|
||||||
|
add(new RemoveSubscriptionInfoMarshaller());
|
||||||
|
add(new ReplayCommandMarshaller());
|
||||||
|
add(new ResponseMarshaller());
|
||||||
|
add(new SessionIdMarshaller());
|
||||||
|
add(new SessionInfoMarshaller());
|
||||||
|
add(new ShutdownInfoMarshaller());
|
||||||
|
add(new SubscriptionInfoMarshaller());
|
||||||
|
add(new TransactionInfoMarshaller());
|
||||||
|
add(new WireFormatInfoMarshaller());
|
||||||
|
add(new XATransactionIdMarshaller());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
static private void add(DataStreamMarshaller dsm) {
|
||||||
|
marshaller[dsm.getDataStructureType()] = dsm;
|
||||||
|
}
|
||||||
|
|
||||||
|
static public DataStreamMarshaller[] createMarshallerMap(OpenWireFormat wireFormat) {
|
||||||
|
return marshaller;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,162 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for MessageAckMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class MessageAckMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return MessageAck.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new MessageAck();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
MessageAck info = (MessageAck)o;
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setAckType(dataIn.readByte());
|
||||||
|
info.setFirstMessageId((org.apache.activemq.command.MessageId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setLastMessageId((org.apache.activemq.command.MessageId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setMessageCount(dataIn.readInt());
|
||||||
|
info.setPoisonCause((java.lang.Throwable) tightUnmarsalThrowable(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
MessageAck info = (MessageAck)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getTransactionId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConsumerId(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getFirstMessageId(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getLastMessageId(), bs);
|
||||||
|
rc += tightMarshalThrowable1(wireFormat, info.getPoisonCause(), bs);
|
||||||
|
|
||||||
|
return rc + 5;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
MessageAck info = (MessageAck)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getTransactionId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConsumerId(), dataOut, bs);
|
||||||
|
dataOut.writeByte(info.getAckType());
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getFirstMessageId(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getLastMessageId(), dataOut, bs);
|
||||||
|
dataOut.writeInt(info.getMessageCount());
|
||||||
|
tightMarshalThrowable2(wireFormat, info.getPoisonCause(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
MessageAck info = (MessageAck)o;
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setAckType(dataIn.readByte());
|
||||||
|
info.setFirstMessageId((org.apache.activemq.command.MessageId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setLastMessageId((org.apache.activemq.command.MessageId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setMessageCount(dataIn.readInt());
|
||||||
|
info.setPoisonCause((java.lang.Throwable) looseUnmarsalThrowable(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
MessageAck info = (MessageAck)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getTransactionId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConsumerId(), dataOut);
|
||||||
|
dataOut.writeByte(info.getAckType());
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getFirstMessageId(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getLastMessageId(), dataOut);
|
||||||
|
dataOut.writeInt(info.getMessageCount());
|
||||||
|
looseMarshalThrowable(wireFormat, info.getPoisonCause(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,143 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for MessageDispatchMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class MessageDispatchMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return MessageDispatch.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new MessageDispatch();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
MessageDispatch info = (MessageDispatch)o;
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setMessage((org.apache.activemq.command.Message) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setRedeliveryCounter(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
MessageDispatch info = (MessageDispatch)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConsumerId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getMessage(), bs);
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
MessageDispatch info = (MessageDispatch)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConsumerId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getMessage(), dataOut, bs);
|
||||||
|
dataOut.writeInt(info.getRedeliveryCounter());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
MessageDispatch info = (MessageDispatch)o;
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setMessage((org.apache.activemq.command.Message) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setRedeliveryCounter(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
MessageDispatch info = (MessageDispatch)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConsumerId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getMessage(), dataOut);
|
||||||
|
dataOut.writeInt(info.getRedeliveryCounter());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,144 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for MessageDispatchNotificationMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class MessageDispatchNotificationMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return MessageDispatchNotification.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new MessageDispatchNotification();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
MessageDispatchNotification info = (MessageDispatchNotification)o;
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setDeliverySequenceId(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setMessageId((org.apache.activemq.command.MessageId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
MessageDispatchNotification info = (MessageDispatchNotification)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConsumerId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getDeliverySequenceId(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getMessageId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
MessageDispatchNotification info = (MessageDispatchNotification)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConsumerId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getDeliverySequenceId(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getMessageId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
MessageDispatchNotification info = (MessageDispatchNotification)o;
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setDeliverySequenceId(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setMessageId((org.apache.activemq.command.MessageId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
MessageDispatchNotification info = (MessageDispatchNotification)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConsumerId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getDeliverySequenceId(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getMessageId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,144 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for MessageIdMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class MessageIdMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return MessageId.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new MessageId();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
MessageId info = (MessageId)o;
|
||||||
|
info.setTextView(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setProducerId((org.apache.activemq.command.ProducerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setProducerSequenceId(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setBrokerSequenceId(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
MessageId info = (MessageId)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getTextView(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getProducerId(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getProducerSequenceId(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getBrokerSequenceId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
MessageId info = (MessageId)o;
|
||||||
|
tightMarshalString2(info.getTextView(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getProducerId(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getProducerSequenceId(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getBrokerSequenceId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
MessageId info = (MessageId)o;
|
||||||
|
info.setTextView(looseUnmarshalString(dataIn));
|
||||||
|
info.setProducerId((org.apache.activemq.command.ProducerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setProducerSequenceId(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setBrokerSequenceId(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
MessageId info = (MessageId)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getTextView(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getProducerId(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getProducerSequenceId(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getBrokerSequenceId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,316 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for MessageMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public abstract class MessageMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
Message info = (Message)o;
|
||||||
|
|
||||||
|
info.beforeUnmarshall(wireFormat);
|
||||||
|
|
||||||
|
info.setProducerId((org.apache.activemq.command.ProducerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setOriginalDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setMessageId((org.apache.activemq.command.MessageId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setOriginalTransactionId((org.apache.activemq.command.TransactionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setGroupID(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setGroupSequence(dataIn.readInt());
|
||||||
|
info.setCorrelationId(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setPersistent(bs.readBoolean());
|
||||||
|
info.setExpiration(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setPriority(dataIn.readByte());
|
||||||
|
info.setReplyTo((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setTimestamp(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setType(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setContent(tightUnmarshalByteSequence(dataIn, bs));
|
||||||
|
info.setMarshalledProperties(tightUnmarshalByteSequence(dataIn, bs));
|
||||||
|
info.setDataStructure((org.apache.activemq.command.DataStructure) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setTargetConsumerId((org.apache.activemq.command.ConsumerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setCompressed(bs.readBoolean());
|
||||||
|
info.setRedeliveryCounter(dataIn.readInt());
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
info.setArrival(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setUserID(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setRecievedByDFBridge(bs.readBoolean());
|
||||||
|
info.setDroppable(bs.readBoolean());
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setCluster(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setCluster(null);
|
||||||
|
}
|
||||||
|
info.setBrokerInTime(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setBrokerOutTime(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setJMSXGroupFirstForConsumer(bs.readBoolean());
|
||||||
|
|
||||||
|
info.afterUnmarshall(wireFormat);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
Message info = (Message)o;
|
||||||
|
|
||||||
|
info.beforeMarshall(wireFormat);
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getProducerId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getTransactionId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getOriginalDestination(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getMessageId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getOriginalTransactionId(), bs);
|
||||||
|
rc += tightMarshalString1(info.getGroupID(), bs);
|
||||||
|
rc += tightMarshalString1(info.getCorrelationId(), bs);
|
||||||
|
bs.writeBoolean(info.isPersistent());
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getExpiration(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getReplyTo(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getTimestamp(), bs);
|
||||||
|
rc += tightMarshalString1(info.getType(), bs);
|
||||||
|
rc += tightMarshalByteSequence1(info.getContent(), bs);
|
||||||
|
rc += tightMarshalByteSequence1(info.getMarshalledProperties(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getDataStructure(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getTargetConsumerId(), bs);
|
||||||
|
bs.writeBoolean(info.isCompressed());
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getBrokerPath(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getArrival(), bs);
|
||||||
|
rc += tightMarshalString1(info.getUserID(), bs);
|
||||||
|
bs.writeBoolean(info.isRecievedByDFBridge());
|
||||||
|
bs.writeBoolean(info.isDroppable());
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getCluster(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getBrokerInTime(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getBrokerOutTime(), bs);
|
||||||
|
bs.writeBoolean(info.isJMSXGroupFirstForConsumer());
|
||||||
|
|
||||||
|
return rc + 9;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
Message info = (Message)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getProducerId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getTransactionId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getOriginalDestination(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getMessageId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getOriginalTransactionId(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getGroupID(), dataOut, bs);
|
||||||
|
dataOut.writeInt(info.getGroupSequence());
|
||||||
|
tightMarshalString2(info.getCorrelationId(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalLong2(wireFormat, info.getExpiration(), dataOut, bs);
|
||||||
|
dataOut.writeByte(info.getPriority());
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getReplyTo(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getTimestamp(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getType(), dataOut, bs);
|
||||||
|
tightMarshalByteSequence2(info.getContent(), dataOut, bs);
|
||||||
|
tightMarshalByteSequence2(info.getMarshalledProperties(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getDataStructure(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getTargetConsumerId(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
dataOut.writeInt(info.getRedeliveryCounter());
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getBrokerPath(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getArrival(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getUserID(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
bs.readBoolean();
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getCluster(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getBrokerInTime(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getBrokerOutTime(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
|
||||||
|
info.afterMarshall(wireFormat);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
Message info = (Message)o;
|
||||||
|
|
||||||
|
info.beforeUnmarshall(wireFormat);
|
||||||
|
|
||||||
|
info.setProducerId((org.apache.activemq.command.ProducerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setOriginalDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setMessageId((org.apache.activemq.command.MessageId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setOriginalTransactionId((org.apache.activemq.command.TransactionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setGroupID(looseUnmarshalString(dataIn));
|
||||||
|
info.setGroupSequence(dataIn.readInt());
|
||||||
|
info.setCorrelationId(looseUnmarshalString(dataIn));
|
||||||
|
info.setPersistent(dataIn.readBoolean());
|
||||||
|
info.setExpiration(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setPriority(dataIn.readByte());
|
||||||
|
info.setReplyTo((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setTimestamp(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setType(looseUnmarshalString(dataIn));
|
||||||
|
info.setContent(looseUnmarshalByteSequence(dataIn));
|
||||||
|
info.setMarshalledProperties(looseUnmarshalByteSequence(dataIn));
|
||||||
|
info.setDataStructure((org.apache.activemq.command.DataStructure) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setTargetConsumerId((org.apache.activemq.command.ConsumerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setCompressed(dataIn.readBoolean());
|
||||||
|
info.setRedeliveryCounter(dataIn.readInt());
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
info.setArrival(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setUserID(looseUnmarshalString(dataIn));
|
||||||
|
info.setRecievedByDFBridge(dataIn.readBoolean());
|
||||||
|
info.setDroppable(dataIn.readBoolean());
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setCluster(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setCluster(null);
|
||||||
|
}
|
||||||
|
info.setBrokerInTime(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setBrokerOutTime(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setJMSXGroupFirstForConsumer(dataIn.readBoolean());
|
||||||
|
|
||||||
|
info.afterUnmarshall(wireFormat);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
Message info = (Message)o;
|
||||||
|
|
||||||
|
info.beforeMarshall(wireFormat);
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getProducerId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getTransactionId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getOriginalDestination(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getMessageId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getOriginalTransactionId(), dataOut);
|
||||||
|
looseMarshalString(info.getGroupID(), dataOut);
|
||||||
|
dataOut.writeInt(info.getGroupSequence());
|
||||||
|
looseMarshalString(info.getCorrelationId(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isPersistent());
|
||||||
|
looseMarshalLong(wireFormat, info.getExpiration(), dataOut);
|
||||||
|
dataOut.writeByte(info.getPriority());
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getReplyTo(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getTimestamp(), dataOut);
|
||||||
|
looseMarshalString(info.getType(), dataOut);
|
||||||
|
looseMarshalByteSequence(wireFormat, info.getContent(), dataOut);
|
||||||
|
looseMarshalByteSequence(wireFormat, info.getMarshalledProperties(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getDataStructure(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getTargetConsumerId(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isCompressed());
|
||||||
|
dataOut.writeInt(info.getRedeliveryCounter());
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getBrokerPath(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getArrival(), dataOut);
|
||||||
|
looseMarshalString(info.getUserID(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isRecievedByDFBridge());
|
||||||
|
dataOut.writeBoolean(info.isDroppable());
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getCluster(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getBrokerInTime(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getBrokerOutTime(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isJMSXGroupFirstForConsumer());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,149 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for MessagePullMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class MessagePullMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return MessagePull.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new MessagePull();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
MessagePull info = (MessagePull)o;
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setTimeout(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setCorrelationId(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setMessageId((org.apache.activemq.command.MessageId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
MessagePull info = (MessagePull)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConsumerId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getTimeout(), bs);
|
||||||
|
rc += tightMarshalString1(info.getCorrelationId(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getMessageId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
MessagePull info = (MessagePull)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConsumerId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getTimeout(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getCorrelationId(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getMessageId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
MessagePull info = (MessagePull)o;
|
||||||
|
info.setConsumerId((org.apache.activemq.command.ConsumerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setTimeout(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setCorrelationId(looseUnmarshalString(dataIn));
|
||||||
|
info.setMessageId((org.apache.activemq.command.MessageId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
MessagePull info = (MessagePull)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConsumerId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getTimeout(), dataOut);
|
||||||
|
looseMarshalString(info.getCorrelationId(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getMessageId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,137 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for NetworkBridgeFilterMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class NetworkBridgeFilterMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return NetworkBridgeFilter.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new NetworkBridgeFilter();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
NetworkBridgeFilter info = (NetworkBridgeFilter)o;
|
||||||
|
info.setNetworkBrokerId((org.apache.activemq.command.BrokerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setMessageTTL(dataIn.readInt());
|
||||||
|
info.setConsumerTTL(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
NetworkBridgeFilter info = (NetworkBridgeFilter)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getNetworkBrokerId(), bs);
|
||||||
|
|
||||||
|
return rc + 8;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
NetworkBridgeFilter info = (NetworkBridgeFilter)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getNetworkBrokerId(), dataOut, bs);
|
||||||
|
dataOut.writeInt(info.getMessageTTL());
|
||||||
|
dataOut.writeInt(info.getConsumerTTL());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
NetworkBridgeFilter info = (NetworkBridgeFilter)o;
|
||||||
|
info.setNetworkBrokerId((org.apache.activemq.command.BrokerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setMessageTTL(dataIn.readInt());
|
||||||
|
info.setConsumerTTL(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
NetworkBridgeFilter info = (NetworkBridgeFilter)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getNetworkBrokerId(), dataOut);
|
||||||
|
dataOut.writeInt(info.getMessageTTL());
|
||||||
|
dataOut.writeInt(info.getConsumerTTL());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,133 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for PartialCommandMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class PartialCommandMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return PartialCommand.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new PartialCommand();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
PartialCommand info = (PartialCommand)o;
|
||||||
|
info.setCommandId(dataIn.readInt());
|
||||||
|
info.setData(tightUnmarshalByteArray(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
PartialCommand info = (PartialCommand)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalByteArray1(info.getData(), bs);
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
PartialCommand info = (PartialCommand)o;
|
||||||
|
dataOut.writeInt(info.getCommandId());
|
||||||
|
tightMarshalByteArray2(info.getData(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
PartialCommand info = (PartialCommand)o;
|
||||||
|
info.setCommandId(dataIn.readInt());
|
||||||
|
info.setData(looseUnmarshalByteArray(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
PartialCommand info = (PartialCommand)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
dataOut.writeInt(info.getCommandId());
|
||||||
|
looseMarshalByteArray(wireFormat, info.getData(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,133 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ProducerAckMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ProducerAckMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ProducerAck.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ProducerAck();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ProducerAck info = (ProducerAck)o;
|
||||||
|
info.setProducerId((org.apache.activemq.command.ProducerId) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setSize(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ProducerAck info = (ProducerAck)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getProducerId(), bs);
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ProducerAck info = (ProducerAck)o;
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getProducerId(), dataOut, bs);
|
||||||
|
dataOut.writeInt(info.getSize());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ProducerAck info = (ProducerAck)o;
|
||||||
|
info.setProducerId((org.apache.activemq.command.ProducerId) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setSize(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ProducerAck info = (ProducerAck)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getProducerId(), dataOut);
|
||||||
|
dataOut.writeInt(info.getSize());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,139 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ProducerIdMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ProducerIdMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ProducerId.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ProducerId();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ProducerId info = (ProducerId)o;
|
||||||
|
info.setConnectionId(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setValue(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
info.setSessionId(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ProducerId info = (ProducerId)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getConnectionId(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getValue(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getSessionId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ProducerId info = (ProducerId)o;
|
||||||
|
tightMarshalString2(info.getConnectionId(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getValue(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getSessionId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ProducerId info = (ProducerId)o;
|
||||||
|
info.setConnectionId(looseUnmarshalString(dataIn));
|
||||||
|
info.setValue(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
info.setSessionId(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ProducerId info = (ProducerId)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getConnectionId(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getValue(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getSessionId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,170 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ProducerInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ProducerInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ProducerInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ProducerInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ProducerInfo info = (ProducerInfo)o;
|
||||||
|
info.setProducerId((org.apache.activemq.command.ProducerId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
if (bs.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) tightUnmarsalNestedObject(wireFormat,dataIn, bs);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
info.setDispatchAsync(bs.readBoolean());
|
||||||
|
info.setWindowSize(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ProducerInfo info = (ProducerInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getProducerId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc += tightMarshalObjectArray1(wireFormat, info.getBrokerPath(), bs);
|
||||||
|
bs.writeBoolean(info.isDispatchAsync());
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ProducerInfo info = (ProducerInfo)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getProducerId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
tightMarshalObjectArray2(wireFormat, info.getBrokerPath(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
dataOut.writeInt(info.getWindowSize());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ProducerInfo info = (ProducerInfo)o;
|
||||||
|
info.setProducerId((org.apache.activemq.command.ProducerId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
|
||||||
|
if (dataIn.readBoolean()) {
|
||||||
|
short size = dataIn.readShort();
|
||||||
|
org.apache.activemq.command.BrokerId value[] = new org.apache.activemq.command.BrokerId[size];
|
||||||
|
for( int i=0; i < size; i++ ) {
|
||||||
|
value[i] = (org.apache.activemq.command.BrokerId) looseUnmarsalNestedObject(wireFormat,dataIn);
|
||||||
|
}
|
||||||
|
info.setBrokerPath(value);
|
||||||
|
}
|
||||||
|
else {
|
||||||
|
info.setBrokerPath(null);
|
||||||
|
}
|
||||||
|
info.setDispatchAsync(dataIn.readBoolean());
|
||||||
|
info.setWindowSize(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ProducerInfo info = (ProducerInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getProducerId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
looseMarshalObjectArray(wireFormat, info.getBrokerPath(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isDispatchAsync());
|
||||||
|
dataOut.writeInt(info.getWindowSize());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,134 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for RemoveInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class RemoveInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return RemoveInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new RemoveInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
RemoveInfo info = (RemoveInfo)o;
|
||||||
|
info.setObjectId((org.apache.activemq.command.DataStructure) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setLastDeliveredSequenceId(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
RemoveInfo info = (RemoveInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getObjectId(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getLastDeliveredSequenceId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
RemoveInfo info = (RemoveInfo)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getObjectId(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getLastDeliveredSequenceId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
RemoveInfo info = (RemoveInfo)o;
|
||||||
|
info.setObjectId((org.apache.activemq.command.DataStructure) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setLastDeliveredSequenceId(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
RemoveInfo info = (RemoveInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getObjectId(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getLastDeliveredSequenceId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,139 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for RemoveSubscriptionInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class RemoveSubscriptionInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return RemoveSubscriptionInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new RemoveSubscriptionInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
RemoveSubscriptionInfo info = (RemoveSubscriptionInfo)o;
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setSubcriptionName(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setClientId(tightUnmarshalString(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
RemoveSubscriptionInfo info = (RemoveSubscriptionInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConnectionId(), bs);
|
||||||
|
rc += tightMarshalString1(info.getSubcriptionName(), bs);
|
||||||
|
rc += tightMarshalString1(info.getClientId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
RemoveSubscriptionInfo info = (RemoveSubscriptionInfo)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConnectionId(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getSubcriptionName(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getClientId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
RemoveSubscriptionInfo info = (RemoveSubscriptionInfo)o;
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setSubcriptionName(looseUnmarshalString(dataIn));
|
||||||
|
info.setClientId(looseUnmarshalString(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
RemoveSubscriptionInfo info = (RemoveSubscriptionInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConnectionId(), dataOut);
|
||||||
|
looseMarshalString(info.getSubcriptionName(), dataOut);
|
||||||
|
looseMarshalString(info.getClientId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,132 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ReplayCommandMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ReplayCommandMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ReplayCommand.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ReplayCommand();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
ReplayCommand info = (ReplayCommand)o;
|
||||||
|
info.setFirstNakNumber(dataIn.readInt());
|
||||||
|
info.setLastNakNumber(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
ReplayCommand info = (ReplayCommand)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 8;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
ReplayCommand info = (ReplayCommand)o;
|
||||||
|
dataOut.writeInt(info.getFirstNakNumber());
|
||||||
|
dataOut.writeInt(info.getLastNakNumber());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
ReplayCommand info = (ReplayCommand)o;
|
||||||
|
info.setFirstNakNumber(dataIn.readInt());
|
||||||
|
info.setLastNakNumber(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
ReplayCommand info = (ReplayCommand)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
dataOut.writeInt(info.getFirstNakNumber());
|
||||||
|
dataOut.writeInt(info.getLastNakNumber());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,128 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ResponseMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ResponseMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return Response.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new Response();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
Response info = (Response)o;
|
||||||
|
info.setCorrelationId(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
Response info = (Response)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
Response info = (Response)o;
|
||||||
|
dataOut.writeInt(info.getCorrelationId());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
Response info = (Response)o;
|
||||||
|
info.setCorrelationId(dataIn.readInt());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
Response info = (Response)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
dataOut.writeInt(info.getCorrelationId());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,134 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for SessionIdMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class SessionIdMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return SessionId.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new SessionId();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
SessionId info = (SessionId)o;
|
||||||
|
info.setConnectionId(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setValue(tightUnmarshalLong(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
SessionId info = (SessionId)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getConnectionId(), bs);
|
||||||
|
rc+=tightMarshalLong1(wireFormat, info.getValue(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
SessionId info = (SessionId)o;
|
||||||
|
tightMarshalString2(info.getConnectionId(), dataOut, bs);
|
||||||
|
tightMarshalLong2(wireFormat, info.getValue(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
SessionId info = (SessionId)o;
|
||||||
|
info.setConnectionId(looseUnmarshalString(dataIn));
|
||||||
|
info.setValue(looseUnmarshalLong(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
SessionId info = (SessionId)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getConnectionId(), dataOut);
|
||||||
|
looseMarshalLong(wireFormat, info.getValue(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for SessionInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class SessionInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return SessionInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new SessionInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
SessionInfo info = (SessionInfo)o;
|
||||||
|
info.setSessionId((org.apache.activemq.command.SessionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
SessionInfo info = (SessionInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getSessionId(), bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
SessionInfo info = (SessionInfo)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getSessionId(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
SessionInfo info = (SessionInfo)o;
|
||||||
|
info.setSessionId((org.apache.activemq.command.SessionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
SessionInfo info = (SessionInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getSessionId(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,114 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for ShutdownInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class ShutdownInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return ShutdownInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new ShutdownInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,154 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for SubscriptionInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class SubscriptionInfoMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return SubscriptionInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new SubscriptionInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
SubscriptionInfo info = (SubscriptionInfo)o;
|
||||||
|
info.setClientId(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setSelector(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setSubcriptionName(tightUnmarshalString(dataIn, bs));
|
||||||
|
info.setSubscribedDestination((org.apache.activemq.command.ActiveMQDestination) tightUnmarsalNestedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setNoLocal(bs.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
SubscriptionInfo info = (SubscriptionInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalString1(info.getClientId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getDestination(), bs);
|
||||||
|
rc += tightMarshalString1(info.getSelector(), bs);
|
||||||
|
rc += tightMarshalString1(info.getSubcriptionName(), bs);
|
||||||
|
rc += tightMarshalNestedObject1(wireFormat, (DataStructure)info.getSubscribedDestination(), bs);
|
||||||
|
bs.writeBoolean(info.isNoLocal());
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
SubscriptionInfo info = (SubscriptionInfo)o;
|
||||||
|
tightMarshalString2(info.getClientId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getDestination(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getSelector(), dataOut, bs);
|
||||||
|
tightMarshalString2(info.getSubcriptionName(), dataOut, bs);
|
||||||
|
tightMarshalNestedObject2(wireFormat, (DataStructure)info.getSubscribedDestination(), dataOut, bs);
|
||||||
|
bs.readBoolean();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
SubscriptionInfo info = (SubscriptionInfo)o;
|
||||||
|
info.setClientId(looseUnmarshalString(dataIn));
|
||||||
|
info.setDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setSelector(looseUnmarshalString(dataIn));
|
||||||
|
info.setSubcriptionName(looseUnmarshalString(dataIn));
|
||||||
|
info.setSubscribedDestination((org.apache.activemq.command.ActiveMQDestination) looseUnmarsalNestedObject(wireFormat, dataIn));
|
||||||
|
info.setNoLocal(dataIn.readBoolean());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
SubscriptionInfo info = (SubscriptionInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalString(info.getClientId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getDestination(), dataOut);
|
||||||
|
looseMarshalString(info.getSelector(), dataOut);
|
||||||
|
looseMarshalString(info.getSubcriptionName(), dataOut);
|
||||||
|
looseMarshalNestedObject(wireFormat, (DataStructure)info.getSubscribedDestination(), dataOut);
|
||||||
|
dataOut.writeBoolean(info.isNoLocal());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,99 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for TransactionIdMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public abstract class TransactionIdMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
|
||||||
|
return rc + 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,138 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for TransactionInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class TransactionInfoMarshaller extends BaseCommandMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return TransactionInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new TransactionInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
TransactionInfo info = (TransactionInfo)o;
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) tightUnmarsalCachedObject(wireFormat, dataIn, bs));
|
||||||
|
info.setType(dataIn.readByte());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
TransactionInfo info = (TransactionInfo)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getConnectionId(), bs);
|
||||||
|
rc += tightMarshalCachedObject1(wireFormat, (DataStructure)info.getTransactionId(), bs);
|
||||||
|
|
||||||
|
return rc + 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
TransactionInfo info = (TransactionInfo)o;
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getConnectionId(), dataOut, bs);
|
||||||
|
tightMarshalCachedObject2(wireFormat, (DataStructure)info.getTransactionId(), dataOut, bs);
|
||||||
|
dataOut.writeByte(info.getType());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
TransactionInfo info = (TransactionInfo)o;
|
||||||
|
info.setConnectionId((org.apache.activemq.command.ConnectionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setTransactionId((org.apache.activemq.command.TransactionId) looseUnmarsalCachedObject(wireFormat, dataIn));
|
||||||
|
info.setType(dataIn.readByte());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
TransactionInfo info = (TransactionInfo)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getConnectionId(), dataOut);
|
||||||
|
looseMarshalCachedObject(wireFormat, (DataStructure)info.getTransactionId(), dataOut);
|
||||||
|
dataOut.writeByte(info.getType());
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,154 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for WireFormatInfoMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class WireFormatInfoMarshaller extends BaseDataStreamMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return WireFormatInfo.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new WireFormatInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
WireFormatInfo info = (WireFormatInfo)o;
|
||||||
|
|
||||||
|
info.beforeUnmarshall(wireFormat);
|
||||||
|
|
||||||
|
info.setMagic(tightUnmarshalConstByteArray(dataIn, bs, 8));
|
||||||
|
info.setVersion(dataIn.readInt());
|
||||||
|
info.setMarshalledProperties(tightUnmarshalByteSequence(dataIn, bs));
|
||||||
|
|
||||||
|
info.afterUnmarshall(wireFormat);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
WireFormatInfo info = (WireFormatInfo)o;
|
||||||
|
|
||||||
|
info.beforeMarshall(wireFormat);
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalConstByteArray1(info.getMagic(), bs, 8);
|
||||||
|
rc += tightMarshalByteSequence1(info.getMarshalledProperties(), bs);
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
WireFormatInfo info = (WireFormatInfo)o;
|
||||||
|
tightMarshalConstByteArray2(info.getMagic(), dataOut, bs, 8);
|
||||||
|
dataOut.writeInt(info.getVersion());
|
||||||
|
tightMarshalByteSequence2(info.getMarshalledProperties(), dataOut, bs);
|
||||||
|
|
||||||
|
info.afterMarshall(wireFormat);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
WireFormatInfo info = (WireFormatInfo)o;
|
||||||
|
|
||||||
|
info.beforeUnmarshall(wireFormat);
|
||||||
|
|
||||||
|
info.setMagic(looseUnmarshalConstByteArray(dataIn, 8));
|
||||||
|
info.setVersion(dataIn.readInt());
|
||||||
|
info.setMarshalledProperties(looseUnmarshalByteSequence(dataIn));
|
||||||
|
|
||||||
|
info.afterUnmarshall(wireFormat);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
WireFormatInfo info = (WireFormatInfo)o;
|
||||||
|
|
||||||
|
info.beforeMarshall(wireFormat);
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
looseMarshalConstByteArray(wireFormat, info.getMagic(), dataOut, 8);
|
||||||
|
dataOut.writeInt(info.getVersion());
|
||||||
|
looseMarshalByteSequence(wireFormat, info.getMarshalledProperties(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,138 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.activemq.openwire.v12;
|
||||||
|
|
||||||
|
import java.io.DataInput;
|
||||||
|
import java.io.DataOutput;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.activemq.openwire.*;
|
||||||
|
import org.apache.activemq.command.*;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marshalling code for Open Wire Format for XATransactionIdMarshaller
|
||||||
|
*
|
||||||
|
*
|
||||||
|
* NOTE!: This file is auto generated - do not modify!
|
||||||
|
* if you need to make a change, please see the modify the groovy scripts in the
|
||||||
|
* under src/gram/script and then use maven openwire:generate to regenerate
|
||||||
|
* this file.
|
||||||
|
*
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class XATransactionIdMarshaller extends TransactionIdMarshaller {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the type of Data Structure we marshal
|
||||||
|
* @return short representation of the type data structure
|
||||||
|
*/
|
||||||
|
public byte getDataStructureType() {
|
||||||
|
return XATransactionId.DATA_STRUCTURE_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return a new object instance
|
||||||
|
*/
|
||||||
|
public DataStructure createObject() {
|
||||||
|
return new XATransactionId();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void tightUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn, BooleanStream bs) throws IOException {
|
||||||
|
super.tightUnmarshal(wireFormat, o, dataIn, bs);
|
||||||
|
|
||||||
|
XATransactionId info = (XATransactionId)o;
|
||||||
|
info.setFormatId(dataIn.readInt());
|
||||||
|
info.setGlobalTransactionId(tightUnmarshalByteArray(dataIn, bs));
|
||||||
|
info.setBranchQualifier(tightUnmarshalByteArray(dataIn, bs));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public int tightMarshal1(OpenWireFormat wireFormat, Object o, BooleanStream bs) throws IOException {
|
||||||
|
|
||||||
|
XATransactionId info = (XATransactionId)o;
|
||||||
|
|
||||||
|
int rc = super.tightMarshal1(wireFormat, o, bs);
|
||||||
|
rc += tightMarshalByteArray1(info.getGlobalTransactionId(), bs);
|
||||||
|
rc += tightMarshalByteArray1(info.getBranchQualifier(), bs);
|
||||||
|
|
||||||
|
return rc + 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a object instance to data output stream
|
||||||
|
*
|
||||||
|
* @param o the instance to be marshaled
|
||||||
|
* @param dataOut the output stream
|
||||||
|
* @throws IOException thrown if an error occurs
|
||||||
|
*/
|
||||||
|
public void tightMarshal2(OpenWireFormat wireFormat, Object o, DataOutput dataOut, BooleanStream bs) throws IOException {
|
||||||
|
super.tightMarshal2(wireFormat, o, dataOut, bs);
|
||||||
|
|
||||||
|
XATransactionId info = (XATransactionId)o;
|
||||||
|
dataOut.writeInt(info.getFormatId());
|
||||||
|
tightMarshalByteArray2(info.getGlobalTransactionId(), dataOut, bs);
|
||||||
|
tightMarshalByteArray2(info.getBranchQualifier(), dataOut, bs);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Un-marshal an object instance from the data input stream
|
||||||
|
*
|
||||||
|
* @param o the object to un-marshal
|
||||||
|
* @param dataIn the data input stream to build the object from
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void looseUnmarshal(OpenWireFormat wireFormat, Object o, DataInput dataIn) throws IOException {
|
||||||
|
super.looseUnmarshal(wireFormat, o, dataIn);
|
||||||
|
|
||||||
|
XATransactionId info = (XATransactionId)o;
|
||||||
|
info.setFormatId(dataIn.readInt());
|
||||||
|
info.setGlobalTransactionId(looseUnmarshalByteArray(dataIn));
|
||||||
|
info.setBranchQualifier(looseUnmarshalByteArray(dataIn));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the booleans that this object uses to a BooleanStream
|
||||||
|
*/
|
||||||
|
public void looseMarshal(OpenWireFormat wireFormat, Object o, DataOutput dataOut) throws IOException {
|
||||||
|
|
||||||
|
XATransactionId info = (XATransactionId)o;
|
||||||
|
|
||||||
|
super.looseMarshal(wireFormat, o, dataOut);
|
||||||
|
dataOut.writeInt(info.getFormatId());
|
||||||
|
looseMarshalByteArray(wireFormat, info.getGlobalTransactionId(), dataOut);
|
||||||
|
looseMarshalByteArray(wireFormat, info.getBranchQualifier(), dataOut);
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.activemq.state;
|
package org.apache.activemq.state;
|
||||||
|
|
||||||
import org.apache.activemq.command.BrokerInfo;
|
import org.apache.activemq.command.BrokerInfo;
|
||||||
|
import org.apache.activemq.command.BrokerSubscriptionInfo;
|
||||||
import org.apache.activemq.command.ConnectionControl;
|
import org.apache.activemq.command.ConnectionControl;
|
||||||
import org.apache.activemq.command.ConnectionError;
|
import org.apache.activemq.command.ConnectionError;
|
||||||
import org.apache.activemq.command.ConnectionId;
|
import org.apache.activemq.command.ConnectionId;
|
||||||
|
@ -94,6 +95,8 @@ public interface CommandVisitor {
|
||||||
|
|
||||||
Response processBrokerInfo(BrokerInfo info) throws Exception;
|
Response processBrokerInfo(BrokerInfo info) throws Exception;
|
||||||
|
|
||||||
|
Response processBrokerSubscriptionInfo(BrokerSubscriptionInfo info) throws Exception;
|
||||||
|
|
||||||
Response processRecoverTransactions(TransactionInfo info) throws Exception;
|
Response processRecoverTransactions(TransactionInfo info) throws Exception;
|
||||||
|
|
||||||
Response processForgetTransaction(TransactionInfo info) throws Exception;
|
Response processForgetTransaction(TransactionInfo info) throws Exception;
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.activemq.state;
|
package org.apache.activemq.state;
|
||||||
|
|
||||||
import org.apache.activemq.command.BrokerInfo;
|
import org.apache.activemq.command.BrokerInfo;
|
||||||
|
import org.apache.activemq.command.BrokerSubscriptionInfo;
|
||||||
import org.apache.activemq.command.ConnectionControl;
|
import org.apache.activemq.command.ConnectionControl;
|
||||||
import org.apache.activemq.command.ConnectionError;
|
import org.apache.activemq.command.ConnectionError;
|
||||||
import org.apache.activemq.command.ConnectionId;
|
import org.apache.activemq.command.ConnectionId;
|
||||||
|
@ -46,141 +47,184 @@ import org.apache.activemq.command.WireFormatInfo;
|
||||||
|
|
||||||
public class CommandVisitorAdapter implements CommandVisitor {
|
public class CommandVisitorAdapter implements CommandVisitor {
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddConnection(ConnectionInfo info) throws Exception {
|
public Response processAddConnection(ConnectionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddConsumer(ConsumerInfo info) throws Exception {
|
public Response processAddConsumer(ConsumerInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddDestination(DestinationInfo info) throws Exception {
|
public Response processAddDestination(DestinationInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddProducer(ProducerInfo info) throws Exception {
|
public Response processAddProducer(ProducerInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processAddSession(SessionInfo info) throws Exception {
|
public Response processAddSession(SessionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processBeginTransaction(TransactionInfo info) throws Exception {
|
public Response processBeginTransaction(TransactionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processBrokerInfo(BrokerInfo info) throws Exception {
|
public Response processBrokerInfo(BrokerInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processCommitTransactionOnePhase(TransactionInfo info) throws Exception {
|
public Response processCommitTransactionOnePhase(TransactionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processCommitTransactionTwoPhase(TransactionInfo info) throws Exception {
|
public Response processCommitTransactionTwoPhase(TransactionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processEndTransaction(TransactionInfo info) throws Exception {
|
public Response processEndTransaction(TransactionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processFlush(FlushCommand command) throws Exception {
|
public Response processFlush(FlushCommand command) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processForgetTransaction(TransactionInfo info) throws Exception {
|
public Response processForgetTransaction(TransactionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processKeepAlive(KeepAliveInfo info) throws Exception {
|
public Response processKeepAlive(KeepAliveInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessage(Message send) throws Exception {
|
public Response processMessage(Message send) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessageAck(MessageAck ack) throws Exception {
|
public Response processMessageAck(MessageAck ack) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessageDispatchNotification(MessageDispatchNotification notification)
|
public Response processMessageDispatchNotification(MessageDispatchNotification notification)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessagePull(MessagePull pull) throws Exception {
|
public Response processMessagePull(MessagePull pull) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processPrepareTransaction(TransactionInfo info) throws Exception {
|
public Response processPrepareTransaction(TransactionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processProducerAck(ProducerAck ack) throws Exception {
|
public Response processProducerAck(ProducerAck ack) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRecoverTransactions(TransactionInfo info) throws Exception {
|
public Response processRecoverTransactions(TransactionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveConnection(ConnectionId id, long lastDeliveredSequenceId) throws Exception {
|
public Response processRemoveConnection(ConnectionId id, long lastDeliveredSequenceId) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveConsumer(ConsumerId id, long lastDeliveredSequenceId) throws Exception {
|
public Response processRemoveConsumer(ConsumerId id, long lastDeliveredSequenceId) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveDestination(DestinationInfo info) throws Exception {
|
public Response processRemoveDestination(DestinationInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveProducer(ProducerId id) throws Exception {
|
public Response processRemoveProducer(ProducerId id) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveSession(SessionId id, long lastDeliveredSequenceId) throws Exception {
|
public Response processRemoveSession(SessionId id, long lastDeliveredSequenceId) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRemoveSubscription(RemoveSubscriptionInfo info) throws Exception {
|
public Response processRemoveSubscription(RemoveSubscriptionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processRollbackTransaction(TransactionInfo info) throws Exception {
|
public Response processRollbackTransaction(TransactionInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processShutdown(ShutdownInfo info) throws Exception {
|
public Response processShutdown(ShutdownInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processWireFormat(WireFormatInfo info) throws Exception {
|
public Response processWireFormat(WireFormatInfo info) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processMessageDispatch(MessageDispatch dispatch) throws Exception {
|
public Response processMessageDispatch(MessageDispatch dispatch) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processControlCommand(ControlCommand command) throws Exception {
|
public Response processControlCommand(ControlCommand command) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processConnectionControl(ConnectionControl control) throws Exception {
|
public Response processConnectionControl(ConnectionControl control) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processConnectionError(ConnectionError error) throws Exception {
|
public Response processConnectionError(ConnectionError error) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public Response processConsumerControl(ConsumerControl control) throws Exception {
|
public Response processConsumerControl(ConsumerControl control) throws Exception {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/* (non-Javadoc)
|
||||||
|
* @see org.apache.activemq.state.CommandVisitor#processBrokerSubscriptionInfo(org.apache.activemq.command.BrokerSubscriptionInfo)
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Response processBrokerSubscriptionInfo(BrokerSubscriptionInfo info) throws Exception {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,604 @@
|
||||||
|
/**
|
||||||
|
* 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.activemq.network;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.net.URI;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import javax.jms.Connection;
|
||||||
|
import javax.jms.MessageConsumer;
|
||||||
|
import javax.jms.Session;
|
||||||
|
|
||||||
|
import org.apache.activemq.ActiveMQConnectionFactory;
|
||||||
|
import org.apache.activemq.advisory.AdvisoryBroker;
|
||||||
|
import org.apache.activemq.broker.BrokerService;
|
||||||
|
import org.apache.activemq.broker.ConnectionContext;
|
||||||
|
import org.apache.activemq.broker.TransportConnector;
|
||||||
|
import org.apache.activemq.broker.region.DurableTopicSubscription;
|
||||||
|
import org.apache.activemq.broker.region.Topic;
|
||||||
|
import org.apache.activemq.command.ActiveMQDestination;
|
||||||
|
import org.apache.activemq.command.ActiveMQTopic;
|
||||||
|
import org.apache.activemq.command.CommandTypes;
|
||||||
|
import org.apache.activemq.command.RemoveSubscriptionInfo;
|
||||||
|
import org.apache.activemq.util.SubscriptionKey;
|
||||||
|
import org.apache.activemq.util.Wait;
|
||||||
|
import org.apache.activemq.util.Wait.Condition;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Assume;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
import org.junit.runners.Parameterized.Parameters;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
public class DurableSyncNetworkBridgeTest {
|
||||||
|
protected static final Logger LOG = LoggerFactory.getLogger(DurableSyncNetworkBridgeTest.class);
|
||||||
|
|
||||||
|
protected Connection localConnection;
|
||||||
|
protected Connection remoteConnection;
|
||||||
|
protected BrokerService localBroker;
|
||||||
|
protected BrokerService remoteBroker;
|
||||||
|
protected Session localSession;
|
||||||
|
protected Session remoteSession;
|
||||||
|
protected ActiveMQTopic included;
|
||||||
|
protected ActiveMQTopic excluded;
|
||||||
|
protected String includeTopicName = "include.test.bar";
|
||||||
|
protected String includeTopicName2 = "include.test.bar2";
|
||||||
|
protected String excludeTopicName = "exclude.test.bar";
|
||||||
|
protected String clientId = "clientId";
|
||||||
|
private boolean dynamicOnly = false;
|
||||||
|
private byte remoteBrokerWireFormatVersion = CommandTypes.PROTOCOL_VERSION;
|
||||||
|
public static enum FLOW {FORWARD, REVERSE};
|
||||||
|
|
||||||
|
private BrokerService broker1;
|
||||||
|
private BrokerService broker2;
|
||||||
|
private Session session1;
|
||||||
|
private final FLOW flow;
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public TemporaryFolder tempFolder = new TemporaryFolder(new File("target"));
|
||||||
|
|
||||||
|
@Parameters
|
||||||
|
public static Collection<Object[]> data() {
|
||||||
|
return Arrays.asList(new Object[][] {
|
||||||
|
{FLOW.FORWARD},
|
||||||
|
{FLOW.REVERSE}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public DurableSyncNetworkBridgeTest(final FLOW flow) {
|
||||||
|
this.flow = flow;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
dynamicOnly = false;
|
||||||
|
remoteBrokerWireFormatVersion = CommandTypes.PROTOCOL_VERSION;
|
||||||
|
doSetUp(true, true, tempFolder.newFolder(), tempFolder.newFolder());
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
doTearDown();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void doTearDown() throws Exception {
|
||||||
|
stopLocalBroker();
|
||||||
|
stopRemoteBroker();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void stopLocalBroker() throws Exception {
|
||||||
|
if (localConnection != null) {
|
||||||
|
localConnection.close();
|
||||||
|
}
|
||||||
|
if (localBroker != null) {
|
||||||
|
localBroker.stop();
|
||||||
|
localBroker.waitUntilStopped();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void stopRemoteBroker() throws Exception {
|
||||||
|
if (remoteConnection != null) {
|
||||||
|
remoteConnection.close();
|
||||||
|
}
|
||||||
|
if (remoteBroker != null) {
|
||||||
|
remoteBroker.stop();
|
||||||
|
remoteBroker.waitUntilStopped();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRemoveSubscriptionPropagate() throws Exception {
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final String subName = "sub1";
|
||||||
|
MessageConsumer sub1 = session1.createDurableSubscriber(topic, subName);
|
||||||
|
sub1.close();
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
|
||||||
|
removeSubscription(broker1, topic, subName);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRemoveSubscriptionPropegateAfterRestart() throws Exception {
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final String subName = "sub1";
|
||||||
|
MessageConsumer sub1 = session1.createDurableSubscriber(topic, subName);
|
||||||
|
sub1.close();
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
|
||||||
|
restartBrokers(true);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
|
||||||
|
removeSubscription(broker1, topic, subName);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRemoveSubscriptionWithBridgeOffline() throws Exception {
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final String subName = "sub1";
|
||||||
|
MessageConsumer sub1 = session1.createDurableSubscriber(topic, subName);
|
||||||
|
sub1.close();
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
|
||||||
|
doTearDown();
|
||||||
|
restartBroker(broker1, false);
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
removeSubscription(broker1, topic, subName);
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
|
||||||
|
//Test that on successful reconnection of the bridge that
|
||||||
|
//the NC sub will be removed
|
||||||
|
restartBroker(broker2, true);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
restartBroker(broker1, true);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAddAndRemoveSubscriptionWithBridgeOfflineMultiTopics() throws Exception {
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final ActiveMQTopic topic2 = new ActiveMQTopic(includeTopicName2);
|
||||||
|
final String subName = "sub1";
|
||||||
|
MessageConsumer sub1 = session1.createDurableSubscriber(topic, subName);
|
||||||
|
sub1.close();
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
|
||||||
|
doTearDown();
|
||||||
|
restartBroker(broker1, false);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
session1.createDurableSubscriber(topic2, "sub2");
|
||||||
|
removeSubscription(broker1, topic, subName);
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
assertSubscriptionsCount(broker1, topic2, 1);
|
||||||
|
|
||||||
|
//Test that on successful reconnection of the bridge that
|
||||||
|
//the NC sub will be removed for topic1 but will stay for topic2
|
||||||
|
|
||||||
|
//before sync, the old NC should exist
|
||||||
|
restartBroker(broker2, true);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic2, 0);
|
||||||
|
|
||||||
|
//After sync, remove old NC and create one for topic 2
|
||||||
|
restartBroker(broker1, true);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic2, 1);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAddSubscriptionsWithBridgeOffline() throws Exception {
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final ActiveMQTopic topic2 = new ActiveMQTopic(includeTopicName2);
|
||||||
|
final ActiveMQTopic excludeTopic = new ActiveMQTopic(excludeTopicName);
|
||||||
|
final String subName = "sub1";
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
|
||||||
|
doTearDown();
|
||||||
|
restartBroker(broker1, false);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
//add three subs, should only create 2 NC subs because of conduit
|
||||||
|
session1.createDurableSubscriber(topic, subName).close();
|
||||||
|
session1.createDurableSubscriber(topic, "sub2").close();
|
||||||
|
session1.createDurableSubscriber(topic2, "sub3").close();
|
||||||
|
assertSubscriptionsCount(broker1, topic, 2);
|
||||||
|
assertSubscriptionsCount(broker1, topic2, 1);
|
||||||
|
|
||||||
|
restartBrokers(true);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic2, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, excludeTopic, 0);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Using an older version of openwire should not sync but the network bridge
|
||||||
|
* should still start without error
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testAddSubscriptionsWithBridgeOfflineOpenWire11() throws Exception {
|
||||||
|
this.remoteBrokerWireFormatVersion = CommandTypes.PROTOCOL_VERSION_DURABLE_SYNC - 1;
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final String subName = "sub1";
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
|
||||||
|
doTearDown();
|
||||||
|
restartBroker(broker1, false);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
session1.createDurableSubscriber(topic, subName).close();
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
|
||||||
|
//Since we are using an old version of openwire, the NC should
|
||||||
|
//not be added
|
||||||
|
restartBrokers(true);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAddOfflineSubscriptionWithBridgeOfflineDynamicTrue() throws Exception {
|
||||||
|
//set dynamicOnly to true
|
||||||
|
this.dynamicOnly = true;
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final String subName = "sub1";
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
|
||||||
|
doTearDown();
|
||||||
|
restartBroker(broker1, false);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
session1.createDurableSubscriber(topic, subName).close();
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
|
||||||
|
restartBrokers(true);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAddOnlineSubscriptionWithBridgeOfflineDynamicTrue() throws Exception {
|
||||||
|
//set dynamicOnly to true
|
||||||
|
this.dynamicOnly = true;
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final String subName = "sub1";
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
|
||||||
|
doTearDown();
|
||||||
|
restartBroker(broker1, false);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
session1.createDurableSubscriber(topic, subName).close();
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
|
||||||
|
restartBrokers(true);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
//bring online again
|
||||||
|
session1.createDurableSubscriber(topic, subName);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAddAndRemoveSubscriptionsWithBridgeOffline() throws Exception {
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final ActiveMQTopic excludeTopic = new ActiveMQTopic(excludeTopicName);
|
||||||
|
final String subName = "sub1";
|
||||||
|
|
||||||
|
session1.createDurableSubscriber(topic, subName).close();
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
|
||||||
|
doTearDown();
|
||||||
|
restartBroker(broker1, false);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
removeSubscription(broker1, topic, subName);
|
||||||
|
session1.createDurableSubscriber(topic, "sub2").close();
|
||||||
|
assertSubscriptionsCount(broker1, topic, 1);
|
||||||
|
|
||||||
|
restartBrokers(true);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, excludeTopic, 0);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAddOnlineSubscriptionsWithBridgeOffline() throws Exception {
|
||||||
|
Assume.assumeTrue(flow == FLOW.FORWARD);
|
||||||
|
|
||||||
|
final ActiveMQTopic topic = new ActiveMQTopic(includeTopicName);
|
||||||
|
final ActiveMQTopic excludeTopic = new ActiveMQTopic(excludeTopicName);
|
||||||
|
final String subName = "sub1";
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 0);
|
||||||
|
|
||||||
|
doTearDown();
|
||||||
|
restartBrokers(false);
|
||||||
|
|
||||||
|
assertSubscriptionsCount(broker1, topic, 0);
|
||||||
|
|
||||||
|
//create durable that shouldn't be propagated
|
||||||
|
session1.createDurableSubscriber(excludeTopic, "sub-exclude");
|
||||||
|
|
||||||
|
//Add 3 online subs
|
||||||
|
session1.createDurableSubscriber(topic, subName);
|
||||||
|
session1.createDurableSubscriber(topic, "sub2");
|
||||||
|
session1.createDurableSubscriber(topic, "sub3");
|
||||||
|
assertSubscriptionsCount(broker1, topic, 3);
|
||||||
|
|
||||||
|
//Restart brokers and make sure we don't have duplicate NCs created
|
||||||
|
//between the sync command and the online durables that are added over
|
||||||
|
//the consumer advisory
|
||||||
|
restartBrokers(true);
|
||||||
|
|
||||||
|
//Re-create
|
||||||
|
session1.createDurableSubscriber(topic, subName);
|
||||||
|
session1.createDurableSubscriber(topic, "sub2");
|
||||||
|
session1.createDurableSubscriber(topic, "sub3");
|
||||||
|
session1.createDurableSubscriber(excludeTopic, "sub-exclude");
|
||||||
|
|
||||||
|
Thread.sleep(1000);
|
||||||
|
assertNCSubscriptionsCount(broker2, topic, 1);
|
||||||
|
assertNCSubscriptionsCount(broker2, excludeTopic, 0);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void removeSubscription(final BrokerService brokerService, final ActiveMQTopic topic,
|
||||||
|
final String subName) throws Exception {
|
||||||
|
final RemoveSubscriptionInfo info = new RemoveSubscriptionInfo();
|
||||||
|
info.setClientId(clientId);
|
||||||
|
info.setSubscriptionName(subName);
|
||||||
|
|
||||||
|
final ConnectionContext context = new ConnectionContext();
|
||||||
|
context.setBroker(brokerService.getBroker());
|
||||||
|
context.setClientId(clientId);
|
||||||
|
|
||||||
|
brokerService.getBroker().removeSubscription(context, info);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void assertSubscriptionsCount(final BrokerService brokerService,
|
||||||
|
final ActiveMQTopic dest, final int count) throws Exception {
|
||||||
|
assertTrue(Wait.waitFor(new Condition() {
|
||||||
|
@Override
|
||||||
|
public boolean isSatisified() throws Exception {
|
||||||
|
return count == getSubscriptions(brokerService, dest).size();
|
||||||
|
}
|
||||||
|
}, 10000, 500));
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void assertNCSubscriptionsCount(final BrokerService brokerService,
|
||||||
|
final ActiveMQTopic dest, final int count) throws Exception {
|
||||||
|
assertTrue(Wait.waitFor(new Condition() {
|
||||||
|
@Override
|
||||||
|
public boolean isSatisified() throws Exception {
|
||||||
|
return count == getNCSubscriptions(brokerService, dest).size();
|
||||||
|
}
|
||||||
|
}, 10000, 500));
|
||||||
|
}
|
||||||
|
|
||||||
|
protected List<DurableTopicSubscription> getSubscriptions(final BrokerService brokerService,
|
||||||
|
final ActiveMQTopic dest) throws Exception {
|
||||||
|
List<DurableTopicSubscription> subs = new ArrayList<>();
|
||||||
|
Topic destination = (Topic) brokerService.getDestination(dest);
|
||||||
|
for (SubscriptionKey key : destination.getDurableTopicSubs().keySet()) {
|
||||||
|
if (!key.getSubscriptionName().startsWith(DemandForwardingBridge.DURABLE_SUB_PREFIX)) {
|
||||||
|
DurableTopicSubscription sub = destination.getDurableTopicSubs().get(key);
|
||||||
|
if (sub != null) {
|
||||||
|
subs.add(sub);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return subs;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected List<DurableTopicSubscription> getNCSubscriptions(final BrokerService brokerService,
|
||||||
|
final ActiveMQTopic dest) throws Exception {
|
||||||
|
List<DurableTopicSubscription> subs = new ArrayList<>();
|
||||||
|
Topic destination = (Topic) brokerService.getDestination(dest);
|
||||||
|
for (SubscriptionKey key : destination.getDurableTopicSubs().keySet()) {
|
||||||
|
if (key.getSubscriptionName().startsWith(DemandForwardingBridge.DURABLE_SUB_PREFIX)) {
|
||||||
|
DurableTopicSubscription sub = destination.getDurableTopicSubs().get(key);
|
||||||
|
if (sub != null) {
|
||||||
|
subs.add(sub);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return subs;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void restartBroker(BrokerService broker, boolean startNetworkConnector) throws Exception {
|
||||||
|
if (broker.getBrokerName().equals("localBroker")) {
|
||||||
|
restartLocalBroker(startNetworkConnector);
|
||||||
|
} else {
|
||||||
|
restartRemoteBroker();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void restartBrokers(boolean startNetworkConnector) throws Exception {
|
||||||
|
doTearDown();
|
||||||
|
doSetUp(false, startNetworkConnector, localBroker.getDataDirectoryFile(),
|
||||||
|
remoteBroker.getDataDirectoryFile());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void doSetUp(boolean deleteAllMessages, boolean startNetworkConnector, File localDataDir,
|
||||||
|
File remoteDataDir) throws Exception {
|
||||||
|
included = new ActiveMQTopic(includeTopicName);
|
||||||
|
doSetUpRemoteBroker(deleteAllMessages, remoteDataDir);
|
||||||
|
doSetUpLocalBroker(deleteAllMessages, startNetworkConnector, localDataDir);
|
||||||
|
//Give time for advisories to propagate
|
||||||
|
Thread.sleep(1000);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void restartLocalBroker(boolean startNetworkConnector) throws Exception {
|
||||||
|
stopLocalBroker();
|
||||||
|
doSetUpLocalBroker(false, startNetworkConnector, localBroker.getDataDirectoryFile());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void restartRemoteBroker() throws Exception {
|
||||||
|
stopRemoteBroker();
|
||||||
|
doSetUpRemoteBroker(false, remoteBroker.getDataDirectoryFile());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void doSetUpLocalBroker(boolean deleteAllMessages, boolean startNetworkConnector,
|
||||||
|
File dataDir) throws Exception {
|
||||||
|
localBroker = createLocalBroker(dataDir, startNetworkConnector);
|
||||||
|
localBroker.setDeleteAllMessagesOnStartup(deleteAllMessages);
|
||||||
|
localBroker.start();
|
||||||
|
localBroker.waitUntilStarted();
|
||||||
|
URI localURI = localBroker.getVmConnectorURI();
|
||||||
|
ActiveMQConnectionFactory fac = new ActiveMQConnectionFactory(localURI);
|
||||||
|
fac.setAlwaysSyncSend(true);
|
||||||
|
fac.setDispatchAsync(false);
|
||||||
|
localConnection = fac.createConnection();
|
||||||
|
localConnection.setClientID("clientId");
|
||||||
|
localConnection.start();
|
||||||
|
|
||||||
|
if (startNetworkConnector) { // brokerService.setPlugins(new BrokerPlugin[] {new
|
||||||
|
// JavaRuntimeConfigurationPlugin()});
|
||||||
|
// brokerService.setUseVirtualDestSubs(true);
|
||||||
|
// brokerService.setUseVirtualDestSubsOnCreation(isUsevirtualDestinationSubscriptionsOnCreation);
|
||||||
|
Wait.waitFor(new Condition() {
|
||||||
|
@Override
|
||||||
|
public boolean isSatisified() throws Exception {
|
||||||
|
return localBroker.getNetworkConnectors().get(0).activeBridges().size() == 1;
|
||||||
|
}
|
||||||
|
}, 10000, 500);
|
||||||
|
}
|
||||||
|
localSession = localConnection.createSession(false, Session.AUTO_ACKNOWLEDGE);
|
||||||
|
|
||||||
|
if (flow.equals(FLOW.FORWARD)) {
|
||||||
|
broker1 = localBroker;
|
||||||
|
session1 = localSession;
|
||||||
|
} else {
|
||||||
|
broker2 = localBroker;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void doSetUpRemoteBroker(boolean deleteAllMessages, File dataDir) throws Exception {
|
||||||
|
remoteBroker = createRemoteBroker(dataDir);
|
||||||
|
remoteBroker.setDeleteAllMessagesOnStartup(deleteAllMessages);
|
||||||
|
remoteBroker.start();
|
||||||
|
remoteBroker.waitUntilStarted();
|
||||||
|
URI remoteURI = remoteBroker.getVmConnectorURI();
|
||||||
|
ActiveMQConnectionFactory fac = new ActiveMQConnectionFactory(remoteURI);
|
||||||
|
remoteConnection = fac.createConnection();
|
||||||
|
remoteConnection.setClientID("clientId");
|
||||||
|
remoteConnection.start();
|
||||||
|
remoteSession = remoteConnection.createSession(false, Session.AUTO_ACKNOWLEDGE);
|
||||||
|
|
||||||
|
if (flow.equals(FLOW.FORWARD)) {
|
||||||
|
broker2 = remoteBroker;
|
||||||
|
} else {
|
||||||
|
broker1 = remoteBroker;
|
||||||
|
session1 = remoteSession;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected BrokerService createLocalBroker(File dataDir, boolean startNetworkConnector) throws Exception {
|
||||||
|
BrokerService brokerService = new BrokerService();
|
||||||
|
brokerService.setMonitorConnectionSplits(true);
|
||||||
|
brokerService.setDataDirectoryFile(dataDir);
|
||||||
|
brokerService.setBrokerName("localBroker");
|
||||||
|
|
||||||
|
if (startNetworkConnector) {
|
||||||
|
brokerService.addNetworkConnector(configureLocalNetworkConnector());
|
||||||
|
}
|
||||||
|
|
||||||
|
brokerService.addConnector("tcp://localhost:0");
|
||||||
|
|
||||||
|
return brokerService;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected NetworkConnector configureLocalNetworkConnector() throws Exception {
|
||||||
|
List<TransportConnector> transportConnectors = remoteBroker.getTransportConnectors();
|
||||||
|
URI remoteURI = transportConnectors.get(0).getConnectUri();
|
||||||
|
String uri = "static:(" + remoteURI + ")";
|
||||||
|
NetworkConnector connector = new DiscoveryNetworkConnector(new URI(uri));
|
||||||
|
connector.setName("networkConnector");
|
||||||
|
connector.setDynamicOnly(dynamicOnly);
|
||||||
|
connector.setDecreaseNetworkConsumerPriority(false);
|
||||||
|
connector.setConduitSubscriptions(true);
|
||||||
|
connector.setDuplex(true);
|
||||||
|
connector.setStaticBridge(false);
|
||||||
|
connector.setSyncDurableSubs(true);
|
||||||
|
connector.setDynamicallyIncludedDestinations(
|
||||||
|
Lists.<ActiveMQDestination>newArrayList(new ActiveMQTopic("include.test.>")));
|
||||||
|
connector.setExcludedDestinations(
|
||||||
|
Lists.<ActiveMQDestination>newArrayList(new ActiveMQTopic(excludeTopicName)));
|
||||||
|
return connector;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected AdvisoryBroker remoteAdvisoryBroker;
|
||||||
|
|
||||||
|
protected BrokerService createRemoteBroker(File dataDir) throws Exception {
|
||||||
|
BrokerService brokerService = new BrokerService();
|
||||||
|
brokerService.setBrokerName("remoteBroker");
|
||||||
|
brokerService.setUseJmx(false);
|
||||||
|
brokerService.setDataDirectoryFile(dataDir);
|
||||||
|
|
||||||
|
remoteAdvisoryBroker = (AdvisoryBroker) brokerService.getBroker().getAdaptor(AdvisoryBroker.class);
|
||||||
|
|
||||||
|
brokerService.addConnector("tcp://localhost:0?wireFormat.version=" + remoteBrokerWireFormatVersion);
|
||||||
|
|
||||||
|
return brokerService;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
Loading…
Reference in New Issue