This closes #1654
This commit is contained in:
commit
fd2ce26d50
|
@ -343,7 +343,7 @@ public final class StompConnection implements RemotingConnection {
|
|||
|
||||
StompFrame frame = frameHandler.createStompFrame(Stomp.Responses.ERROR);
|
||||
frame.addHeader(Stomp.Headers.Error.MESSAGE, me.getMessage());
|
||||
sendFrame(frame);
|
||||
sendFrame(frame, null);
|
||||
|
||||
destroyed = true;
|
||||
}
|
||||
|
@ -552,7 +552,7 @@ public final class StompConnection implements RemotingConnection {
|
|||
}
|
||||
|
||||
if (reply != null) {
|
||||
sendFrame(reply);
|
||||
sendFrame(reply, null);
|
||||
}
|
||||
|
||||
if (Stomp.Commands.DISCONNECT.equals(cmd)) {
|
||||
|
@ -560,8 +560,8 @@ public final class StompConnection implements RemotingConnection {
|
|||
}
|
||||
}
|
||||
|
||||
public void sendFrame(StompFrame frame) {
|
||||
manager.sendReply(this, frame);
|
||||
public void sendFrame(StompFrame frame, StompPostReceiptFunction function) {
|
||||
manager.sendReply(this, frame, function);
|
||||
}
|
||||
|
||||
public boolean validateUser(final String login, final String pass, final RemotingConnection connection) {
|
||||
|
@ -660,7 +660,7 @@ public final class StompConnection implements RemotingConnection {
|
|||
}
|
||||
}
|
||||
|
||||
void subscribe(String destination,
|
||||
StompPostReceiptFunction subscribe(String destination,
|
||||
String selector,
|
||||
String ack,
|
||||
String id,
|
||||
|
@ -694,7 +694,7 @@ public final class StompConnection implements RemotingConnection {
|
|||
}
|
||||
|
||||
try {
|
||||
manager.subscribe(this, subscriptionID, durableSubscriptionName, destination, selector, ack, noLocal);
|
||||
return manager.subscribe(this, subscriptionID, durableSubscriptionName, destination, selector, ack, noLocal);
|
||||
} catch (ActiveMQStompException e) {
|
||||
throw e;
|
||||
} catch (Exception e) {
|
||||
|
@ -743,7 +743,7 @@ public final class StompConnection implements RemotingConnection {
|
|||
|
||||
//send a ping stomp frame
|
||||
public void ping(StompFrame pingFrame) {
|
||||
manager.sendReply(this, pingFrame);
|
||||
manager.sendReply(this, pingFrame, null);
|
||||
}
|
||||
|
||||
public void physicalSend(StompFrame frame) throws Exception {
|
||||
|
|
|
@ -0,0 +1,21 @@
|
|||
/*
|
||||
* 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.artemis.core.protocol.stomp;
|
||||
|
||||
public interface StompPostReceiptFunction {
|
||||
void afterReceipt();
|
||||
}
|
|
@ -33,9 +33,9 @@ import org.apache.activemq.artemis.api.core.SimpleString;
|
|||
import org.apache.activemq.artemis.api.core.client.ActiveMQClient;
|
||||
import org.apache.activemq.artemis.core.io.IOCallback;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.remoting.CertificateUtil;
|
||||
import org.apache.activemq.artemis.core.remoting.impl.netty.NettyServerConnection;
|
||||
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
|
||||
import org.apache.activemq.artemis.core.remoting.CertificateUtil;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.ServerSession;
|
||||
|
@ -281,7 +281,7 @@ public class StompProtocolManager extends AbstractProtocolManager<StompFrame, St
|
|||
});
|
||||
}
|
||||
|
||||
public void sendReply(final StompConnection connection, final StompFrame frame) {
|
||||
public void sendReply(final StompConnection connection, final StompFrame frame, final StompPostReceiptFunction function) {
|
||||
server.getStorageManager().afterCompleteOperations(new IOCallback() {
|
||||
@Override
|
||||
public void onError(final int errorCode, final String errorMessage) {
|
||||
|
@ -295,7 +295,13 @@ public class StompProtocolManager extends AbstractProtocolManager<StompFrame, St
|
|||
|
||||
@Override
|
||||
public void done() {
|
||||
send(connection, frame);
|
||||
if (frame != null) {
|
||||
send(connection, frame);
|
||||
}
|
||||
|
||||
if (function != null) {
|
||||
function.afterReceipt();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -361,7 +367,7 @@ public class StompProtocolManager extends AbstractProtocolManager<StompFrame, St
|
|||
}
|
||||
// Inner classes -------------------------------------------------
|
||||
|
||||
public void subscribe(StompConnection connection,
|
||||
public StompPostReceiptFunction subscribe(StompConnection connection,
|
||||
String subscriptionID,
|
||||
String durableSubscriptionName,
|
||||
String destination,
|
||||
|
@ -375,7 +381,7 @@ public class StompProtocolManager extends AbstractProtocolManager<StompFrame, St
|
|||
". Either use unique subscription IDs or do not create multiple subscriptions for the same destination");
|
||||
}
|
||||
long consumerID = server.getStorageManager().generateID();
|
||||
stompSession.addSubscription(consumerID, subscriptionID, connection.getClientID(), durableSubscriptionName, destination, selector, ack);
|
||||
return stompSession.addSubscription(consumerID, subscriptionID, connection.getClientID(), durableSubscriptionName, destination, selector, ack);
|
||||
}
|
||||
|
||||
public void unsubscribe(StompConnection connection,
|
||||
|
|
|
@ -242,7 +242,7 @@ public class StompSession implements SessionCallback {
|
|||
StompFrame frame = connection.getFrameHandler().createStompFrame(Stomp.Responses.ERROR);
|
||||
frame.addHeader(Stomp.Headers.CONTENT_TYPE, "text/plain");
|
||||
frame.setBody("consumer with ID " + consumerId + " disconnected by server");
|
||||
connection.sendFrame(frame);
|
||||
connection.sendFrame(frame, null);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -278,7 +278,7 @@ public class StompSession implements SessionCallback {
|
|||
session.commit();
|
||||
}
|
||||
|
||||
public void addSubscription(long consumerID,
|
||||
public StompPostReceiptFunction addSubscription(long consumerID,
|
||||
String subscriptionID,
|
||||
String clientID,
|
||||
String durableSubscriptionName,
|
||||
|
@ -287,13 +287,11 @@ public class StompSession implements SessionCallback {
|
|||
String ack) throws Exception {
|
||||
SimpleString queueName = SimpleString.toSimpleString(destination);
|
||||
boolean pubSub = false;
|
||||
int receiveCredits = consumerCredits;
|
||||
if (ack.equals(Stomp.Headers.Subscribe.AckModeValues.AUTO)) {
|
||||
receiveCredits = -1;
|
||||
}
|
||||
final int receiveCredits = ack.equals(Stomp.Headers.Subscribe.AckModeValues.AUTO) ? -1 : consumerCredits;
|
||||
|
||||
Set<RoutingType> routingTypes = manager.getServer().getAddressInfo(getCoreSession().removePrefix(SimpleString.toSimpleString(destination))).getRoutingTypes();
|
||||
if (routingTypes.size() == 1 && routingTypes.contains(RoutingType.MULTICAST)) {
|
||||
boolean topic = routingTypes.size() == 1 && routingTypes.contains(RoutingType.MULTICAST);
|
||||
if (topic) {
|
||||
// subscribes to a topic
|
||||
pubSub = true;
|
||||
if (durableSubscriptionName != null) {
|
||||
|
@ -308,15 +306,12 @@ public class StompSession implements SessionCallback {
|
|||
queueName = UUIDGenerator.getInstance().generateSimpleStringUUID();
|
||||
session.createQueue(SimpleString.toSimpleString(destination), queueName, SimpleString.toSimpleString(selector), true, false);
|
||||
}
|
||||
session.createConsumer(consumerID, queueName, null, false, false, receiveCredits);
|
||||
} else {
|
||||
session.createConsumer(consumerID, queueName, SimpleString.toSimpleString(selector), false, false, receiveCredits);
|
||||
}
|
||||
|
||||
final ServerConsumer consumer = topic ? session.createConsumer(consumerID, queueName, null, false, false, 0) : session.createConsumer(consumerID, queueName, SimpleString.toSimpleString(selector), false, false, 0);
|
||||
StompSubscription subscription = new StompSubscription(subscriptionID, ack, queueName, pubSub);
|
||||
subscriptions.put(consumerID, subscription);
|
||||
|
||||
session.start();
|
||||
return () -> consumer.receiveCredits(receiveCredits);
|
||||
}
|
||||
|
||||
public boolean unsubscribe(String id, String durableSubscriptionName, String clientID) throws Exception {
|
||||
|
|
|
@ -96,7 +96,7 @@ public abstract class VersionedStompFrameHandler {
|
|||
} else if (Stomp.Commands.ABORT.equals(request.getCommand())) {
|
||||
response = onAbort(request);
|
||||
} else if (Stomp.Commands.SUBSCRIBE.equals(request.getCommand())) {
|
||||
response = onSubscribe(request);
|
||||
return handleSubscribe(request);
|
||||
} else if (Stomp.Commands.UNSUBSCRIBE.equals(request.getCommand())) {
|
||||
response = onUnsubscribe(request);
|
||||
} else if (Stomp.Commands.CONNECT.equals(request.getCommand())) {
|
||||
|
@ -120,6 +120,21 @@ public abstract class VersionedStompFrameHandler {
|
|||
return response;
|
||||
}
|
||||
|
||||
private StompFrame handleSubscribe(StompFrame request) {
|
||||
StompFrame response = null;
|
||||
try {
|
||||
StompPostReceiptFunction postProcessFunction = onSubscribe(request);
|
||||
response = postprocess(request);
|
||||
if (request.hasHeader(Stomp.Headers.RECEIPT_REQUESTED)) {
|
||||
response.addHeader(Stomp.Headers.Response.RECEIPT_ID, request.getHeader(Stomp.Headers.RECEIPT_REQUESTED));
|
||||
}
|
||||
connection.sendFrame(response, postProcessFunction);
|
||||
return null;
|
||||
} catch (ActiveMQStompException e) {
|
||||
return e.getFrame();
|
||||
}
|
||||
|
||||
}
|
||||
public abstract StompFrame onConnect(StompFrame frame);
|
||||
|
||||
public abstract StompFrame onDisconnect(StompFrame frame);
|
||||
|
@ -240,31 +255,22 @@ public abstract class VersionedStompFrameHandler {
|
|||
return response;
|
||||
}
|
||||
|
||||
public StompFrame onSubscribe(StompFrame frame) {
|
||||
StompFrame response = null;
|
||||
try {
|
||||
String destination = getDestination(frame);
|
||||
public StompPostReceiptFunction onSubscribe(StompFrame frame) throws ActiveMQStompException {
|
||||
String destination = getDestination(frame);
|
||||
|
||||
String selector = frame.getHeader(Stomp.Headers.Subscribe.SELECTOR);
|
||||
String ack = frame.getHeader(Stomp.Headers.Subscribe.ACK_MODE);
|
||||
String id = frame.getHeader(Stomp.Headers.Subscribe.ID);
|
||||
String durableSubscriptionName = frame.getHeader(Stomp.Headers.Subscribe.DURABLE_SUBSCRIBER_NAME);
|
||||
if (durableSubscriptionName == null) {
|
||||
durableSubscriptionName = frame.getHeader(Stomp.Headers.Subscribe.DURABLE_SUBSCRIPTION_NAME);
|
||||
}
|
||||
RoutingType routingType = getRoutingType(frame.getHeader(Headers.Subscribe.SUBSCRIPTION_TYPE), frame.getHeader(Headers.Subscribe.DESTINATION));
|
||||
boolean noLocal = false;
|
||||
|
||||
if (frame.hasHeader(Stomp.Headers.Subscribe.NO_LOCAL)) {
|
||||
noLocal = Boolean.parseBoolean(frame.getHeader(Stomp.Headers.Subscribe.NO_LOCAL));
|
||||
}
|
||||
|
||||
connection.subscribe(destination, selector, ack, id, durableSubscriptionName, noLocal, routingType);
|
||||
} catch (ActiveMQStompException e) {
|
||||
response = e.getFrame();
|
||||
String selector = frame.getHeader(Stomp.Headers.Subscribe.SELECTOR);
|
||||
String ack = frame.getHeader(Stomp.Headers.Subscribe.ACK_MODE);
|
||||
String id = frame.getHeader(Stomp.Headers.Subscribe.ID);
|
||||
String durableSubscriptionName = frame.getHeader(Stomp.Headers.Subscribe.DURABLE_SUBSCRIBER_NAME);
|
||||
if (durableSubscriptionName == null) {
|
||||
durableSubscriptionName = frame.getHeader(Stomp.Headers.Subscribe.DURABLE_SUBSCRIPTION_NAME);
|
||||
}
|
||||
|
||||
return response;
|
||||
RoutingType routingType = getRoutingType(frame.getHeader(Headers.Subscribe.SUBSCRIPTION_TYPE), frame.getHeader(Headers.Subscribe.DESTINATION));
|
||||
boolean noLocal = false;
|
||||
if (frame.hasHeader(Stomp.Headers.Subscribe.NO_LOCAL)) {
|
||||
noLocal = Boolean.parseBoolean(frame.getHeader(Stomp.Headers.Subscribe.NO_LOCAL));
|
||||
}
|
||||
return connection.subscribe(destination, selector, ack, id, durableSubscriptionName, noLocal, routingType);
|
||||
}
|
||||
|
||||
public String getDestination(StompFrame request) {
|
||||
|
@ -334,7 +340,7 @@ public abstract class VersionedStompFrameHandler {
|
|||
|
||||
//sends an ERROR frame back to client if possible then close the connection
|
||||
public void onError(ActiveMQStompException e) {
|
||||
this.connection.sendFrame(e.getFrame());
|
||||
this.connection.sendFrame(e.getFrame(), null);
|
||||
connection.destroy();
|
||||
}
|
||||
|
||||
|
|
|
@ -21,8 +21,8 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.activemq.transport.amqp.client.transport.NettyTransport;
|
||||
import org.apache.activemq.transport.amqp.client.transport.NettyTransportFactory;
|
||||
import org.apache.activemq.transport.netty.NettyTransport;
|
||||
import org.apache.activemq.transport.netty.NettyTransportFactory;
|
||||
import org.apache.qpid.proton.amqp.Symbol;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
|
|
@ -33,8 +33,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.activemq.transport.InactivityIOException;
|
||||
import org.apache.activemq.transport.netty.NettyTransport;
|
||||
import org.apache.activemq.transport.amqp.client.sasl.SaslAuthenticator;
|
||||
import org.apache.activemq.transport.amqp.client.transport.NettyTransportListener;
|
||||
import org.apache.activemq.transport.netty.NettyTransportListener;
|
||||
import org.apache.activemq.transport.amqp.client.util.AsyncResult;
|
||||
import org.apache.activemq.transport.amqp.client.util.ClientFuture;
|
||||
import org.apache.activemq.transport.amqp.client.util.IdGenerator;
|
||||
|
@ -80,7 +81,7 @@ public class AmqpConnection extends AmqpAbstractResource<Connection> implements
|
|||
private final AtomicLong sessionIdGenerator = new AtomicLong();
|
||||
private final AtomicLong txIdGenerator = new AtomicLong();
|
||||
private final Collector protonCollector = new CollectorImpl();
|
||||
private final org.apache.activemq.transport.amqp.client.transport.NettyTransport transport;
|
||||
private final NettyTransport transport;
|
||||
private final Transport protonTransport = Transport.Factory.create();
|
||||
|
||||
private final String username;
|
||||
|
@ -109,7 +110,7 @@ public class AmqpConnection extends AmqpAbstractResource<Connection> implements
|
|||
private boolean trace;
|
||||
private boolean noContainerID = false;
|
||||
|
||||
public AmqpConnection(org.apache.activemq.transport.amqp.client.transport.NettyTransport transport, String username, String password) {
|
||||
public AmqpConnection(NettyTransport transport, String username, String password) {
|
||||
setEndpoint(Connection.Factory.create());
|
||||
getEndpoint().collect(protonCollector);
|
||||
|
||||
|
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.transport.amqp.client.transport;
|
||||
package org.apache.activemq.transport.netty;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
|
@ -223,16 +223,16 @@ public class NettyTcpTransport implements NettyTransport {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void send(ByteBuf output) throws IOException {
|
||||
public ChannelFuture send(ByteBuf output) throws IOException {
|
||||
checkConnected();
|
||||
int length = output.readableBytes();
|
||||
if (length == 0) {
|
||||
return;
|
||||
return null;
|
||||
}
|
||||
|
||||
LOG.trace("Attempted write of: {} bytes", length);
|
||||
|
||||
channel.writeAndFlush(output);
|
||||
return channel.writeAndFlush(output);
|
||||
}
|
||||
|
||||
@Override
|
|
@ -14,13 +14,14 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.transport.amqp.client.transport;
|
||||
package org.apache.activemq.transport.netty;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.security.Principal;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
|
||||
/**
|
||||
* Base for all Netty based Transports in this client.
|
||||
|
@ -37,7 +38,7 @@ public interface NettyTransport {
|
|||
|
||||
ByteBuf allocateSendBuffer(int size) throws IOException;
|
||||
|
||||
void send(ByteBuf output) throws IOException;
|
||||
ChannelFuture send(ByteBuf output) throws IOException;
|
||||
|
||||
NettyTransportListener getTransportListener();
|
||||
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.transport.amqp.client.transport;
|
||||
package org.apache.activemq.transport.netty;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.Map;
|
||||
|
@ -65,19 +65,18 @@ public final class NettyTransportFactory {
|
|||
|
||||
NettyTransport result = null;
|
||||
|
||||
switch (remoteURI.getScheme().toLowerCase()) {
|
||||
case "tcp":
|
||||
case "ssl":
|
||||
result = new NettyTcpTransport(remoteURI, transportOptions);
|
||||
break;
|
||||
case "ws":
|
||||
case "wss":
|
||||
result = new NettyWSTransport(remoteURI, transportOptions);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Invalid URI Scheme: " + remoteURI.getScheme());
|
||||
String scheme = remoteURI.getScheme().toLowerCase();
|
||||
if (scheme.startsWith("tcp") || scheme.startsWith("ssl")) {
|
||||
result = new NettyTcpTransport(remoteURI, transportOptions);
|
||||
} else if (scheme.startsWith("ws") || scheme.startsWith("wss")) {
|
||||
// Check for ws subprotocol
|
||||
if (scheme.contains("+")) {
|
||||
transportOptions.setWsSubProtocol(scheme.substring(scheme.indexOf("+") + 1));
|
||||
}
|
||||
result = new NettyWSTransport(remoteURI, transportOptions);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Invalid URI Scheme: " + remoteURI.getScheme());
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.transport.amqp.client.transport;
|
||||
package org.apache.activemq.transport.netty;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.transport.amqp.client.transport;
|
||||
package org.apache.activemq.transport.netty;
|
||||
|
||||
/**
|
||||
* Encapsulates all the TCP Transport options in one configuration object.
|
||||
|
@ -31,6 +31,7 @@ public class NettyTransportOptions implements Cloneable {
|
|||
public static final int DEFAULT_CONNECT_TIMEOUT = 60000;
|
||||
public static final int DEFAULT_TCP_PORT = 5672;
|
||||
public static final boolean DEFAULT_TRACE_BYTES = false;
|
||||
public static final String DEFAULT_WS_SUBPROTOCOL = NettyWSTransport.AMQP_SUB_PROTOCOL;
|
||||
|
||||
public static final NettyTransportOptions INSTANCE = new NettyTransportOptions();
|
||||
|
||||
|
@ -44,6 +45,7 @@ public class NettyTransportOptions implements Cloneable {
|
|||
private boolean tcpNoDelay = DEFAULT_TCP_NO_DELAY;
|
||||
private int defaultTcpPort = DEFAULT_TCP_PORT;
|
||||
private boolean traceBytes = DEFAULT_TRACE_BYTES;
|
||||
private String wsSubProtocol = DEFAULT_WS_SUBPROTOCOL;
|
||||
|
||||
/**
|
||||
* @return the currently set send buffer size in bytes.
|
||||
|
@ -188,6 +190,14 @@ public class NettyTransportOptions implements Cloneable {
|
|||
return false;
|
||||
}
|
||||
|
||||
public String getWsSubProtocol() {
|
||||
return wsSubProtocol;
|
||||
}
|
||||
|
||||
public void setWsSubProtocol(String wsSubProtocol) {
|
||||
this.wsSubProtocol = wsSubProtocol;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NettyTransportOptions clone() {
|
||||
return copyOptions(new NettyTransportOptions());
|
||||
|
@ -202,6 +212,7 @@ public class NettyTransportOptions implements Cloneable {
|
|||
copy.setTcpKeepAlive(isTcpKeepAlive());
|
||||
copy.setTcpNoDelay(isTcpNoDelay());
|
||||
copy.setTrafficClass(getTrafficClass());
|
||||
copy.setWsSubProtocol(getWsSubProtocol());
|
||||
|
||||
return copy;
|
||||
}
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.transport.amqp.client.transport;
|
||||
package org.apache.activemq.transport.netty;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.transport.amqp.client.transport;
|
||||
package org.apache.activemq.transport.netty;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
|
@ -14,12 +14,13 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.transport.amqp.client.transport;
|
||||
package org.apache.activemq.transport.netty;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -50,7 +51,7 @@ public class NettyWSTransport extends NettyTcpTransport {
|
|||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(NettyWSTransport.class);
|
||||
|
||||
private static final String AMQP_SUB_PROTOCOL = "amqp";
|
||||
public static final String AMQP_SUB_PROTOCOL = "amqp";
|
||||
|
||||
/**
|
||||
* Create a new transport instance
|
||||
|
@ -79,16 +80,16 @@ public class NettyWSTransport extends NettyTcpTransport {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void send(ByteBuf output) throws IOException {
|
||||
public ChannelFuture send(ByteBuf output) throws IOException {
|
||||
checkConnected();
|
||||
int length = output.readableBytes();
|
||||
if (length == 0) {
|
||||
return;
|
||||
return null;
|
||||
}
|
||||
|
||||
LOG.trace("Attempted write of: {} bytes", length);
|
||||
|
||||
channel.writeAndFlush(new BinaryWebSocketFrame(output));
|
||||
return channel.writeAndFlush(new BinaryWebSocketFrame(output));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -115,7 +116,7 @@ public class NettyWSTransport extends NettyTcpTransport {
|
|||
|
||||
NettyWebSocketTransportHandler() {
|
||||
handshaker = WebSocketClientHandshakerFactory.newHandshaker(
|
||||
getRemoteLocation(), WebSocketVersion.V13, AMQP_SUB_PROTOCOL,
|
||||
getRemoteLocation(), WebSocketVersion.V13, options.getWsSubProtocol(),
|
||||
true, new DefaultHttpHeaders(), getMaxFrameSize());
|
||||
}
|
||||
|
|
@ -16,7 +16,7 @@
|
|||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.apache.activemq.transport.amqp.client.transport;
|
||||
package org.apache.activemq.transport.netty;
|
||||
|
||||
import javax.net.ssl.SSLEngine;
|
||||
import javax.net.ssl.X509ExtendedKeyManager;
|
|
@ -40,6 +40,7 @@ import static org.apache.activemq.artemis.tests.integration.plugin.MethodCalledV
|
|||
import static org.apache.activemq.artemis.tests.integration.plugin.MethodCalledVerifier.MESSAGE_ACKED;
|
||||
import static org.apache.activemq.artemis.tests.integration.plugin.MethodCalledVerifier.MESSAGE_EXPIRED;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -99,31 +100,38 @@ public class StompPluginTest extends StompTestBase {
|
|||
public void testSendAndReceive() throws Exception {
|
||||
|
||||
// subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
newConn.connect(defUser, defPass);
|
||||
subscribe(newConn, "a-sub");
|
||||
//StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
try {
|
||||
URI uri = new URI("ws+v12.stomp://localhost:61613");
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
subscribe(newConn, "a-sub");
|
||||
|
||||
send(newConn, getQueuePrefix() + getQueueName(), "text/plain", "Hello World 1!");
|
||||
ClientStompFrame frame = newConn.receiveFrame();
|
||||
send(newConn, getQueuePrefix() + getQueueName(), "text/plain", "Hello World 1!");
|
||||
ClientStompFrame frame = newConn.receiveFrame();
|
||||
|
||||
System.out.println("received " + frame);
|
||||
Assert.assertEquals(Stomp.Responses.MESSAGE, frame.getCommand());
|
||||
System.out.println("received " + frame);
|
||||
Assert.assertEquals(Stomp.Responses.MESSAGE, frame.getCommand());
|
||||
|
||||
verifier.validatePluginMethodsAtLeast(1, MESSAGE_ACKED, BEFORE_SEND, AFTER_SEND, BEFORE_MESSAGE_ROUTE, AFTER_MESSAGE_ROUTE, BEFORE_DELIVER,
|
||||
AFTER_DELIVER);
|
||||
verifier.validatePluginMethodsAtLeast(1, MESSAGE_ACKED, BEFORE_SEND, AFTER_SEND, BEFORE_MESSAGE_ROUTE, AFTER_MESSAGE_ROUTE, BEFORE_DELIVER,
|
||||
AFTER_DELIVER);
|
||||
|
||||
|
||||
// unsub
|
||||
unsubscribe(newConn, "a-sub");
|
||||
// unsub
|
||||
unsubscribe(newConn, "a-sub");
|
||||
|
||||
newConn.disconnect();
|
||||
newConn.disconnect();
|
||||
|
||||
verifier.validatePluginMethodsEquals(0, MESSAGE_EXPIRED, BEFORE_DEPLOY_BRIDGE, AFTER_DEPLOY_BRIDGE);
|
||||
verifier.validatePluginMethodsAtLeast(1, AFTER_CREATE_CONNECTION, AFTER_DESTROY_CONNECTION, BEFORE_CREATE_SESSION,
|
||||
AFTER_CREATE_SESSION, BEFORE_CLOSE_SESSION, AFTER_CLOSE_SESSION, BEFORE_CREATE_CONSUMER,
|
||||
AFTER_CREATE_CONSUMER, BEFORE_CLOSE_CONSUMER, AFTER_CLOSE_CONSUMER, BEFORE_CREATE_QUEUE, AFTER_CREATE_QUEUE,
|
||||
MESSAGE_ACKED, BEFORE_SEND, AFTER_SEND, BEFORE_MESSAGE_ROUTE, AFTER_MESSAGE_ROUTE, BEFORE_DELIVER,
|
||||
AFTER_DELIVER);
|
||||
verifier.validatePluginMethodsEquals(0, MESSAGE_EXPIRED, BEFORE_DEPLOY_BRIDGE, AFTER_DEPLOY_BRIDGE);
|
||||
verifier.validatePluginMethodsAtLeast(1, AFTER_CREATE_CONNECTION, AFTER_DESTROY_CONNECTION, BEFORE_CREATE_SESSION,
|
||||
AFTER_CREATE_SESSION, BEFORE_CLOSE_SESSION, AFTER_CLOSE_SESSION, BEFORE_CREATE_CONSUMER,
|
||||
AFTER_CREATE_CONSUMER, BEFORE_CLOSE_CONSUMER, AFTER_CLOSE_CONSUMER, BEFORE_CREATE_QUEUE, AFTER_CREATE_QUEUE,
|
||||
MESSAGE_ACKED, BEFORE_SEND, AFTER_SEND, BEFORE_MESSAGE_ROUTE, AFTER_MESSAGE_ROUTE, BEFORE_DELIVER,
|
||||
AFTER_DELIVER);
|
||||
|
||||
} catch (Throwable e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -16,6 +16,9 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.tests.integration.stomp;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.server.QueueQueryResult;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.ClientStompFrame;
|
||||
|
@ -24,16 +27,24 @@ import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConne
|
|||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class FQQNStompTest extends StompTestBase {
|
||||
|
||||
private StompClientConnection conn;
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> data() {
|
||||
return Arrays.asList(new Object[][]{{"ws+v12.stomp"}, {"tcp+v12.stomp"}});
|
||||
}
|
||||
|
||||
@Override
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
QueueQueryResult result = server.getActiveMQServer().queueQuery(new SimpleString(getQueueName()));
|
||||
assertTrue(result.isExists());
|
||||
System.out.println("address: " + result.getAddress() + " queue " + result.getName());
|
||||
|
@ -51,6 +62,7 @@ public class FQQNStompTest extends StompTestBase {
|
|||
}
|
||||
}
|
||||
} finally {
|
||||
conn.closeTransport();
|
||||
super.tearDown();
|
||||
}
|
||||
}
|
||||
|
@ -83,21 +95,20 @@ public class FQQNStompTest extends StompTestBase {
|
|||
unsubscribe(conn, "sub-01");
|
||||
|
||||
//queue::
|
||||
subscribeQueue(conn, "sub-01", getQueueName() + "\\c\\c");
|
||||
sendJmsMessage("Hello World!");
|
||||
frame = conn.receiveFrame(2000);
|
||||
frame = subscribeQueue(conn, "sub-01", getQueueName() + "\\c\\c");
|
||||
assertNotNull(frame);
|
||||
assertEquals("ERROR", frame.getCommand());
|
||||
assertTrue(frame.getBody().contains(getQueueName()));
|
||||
assertTrue(frame.getBody().contains("not exist"));
|
||||
conn.closeTransport();
|
||||
|
||||
//need reconnect because stomp disconnect on error
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
//:: will subscribe to no queue so no message received.
|
||||
subscribeQueue(conn, "sub-01", "\\c\\c");
|
||||
sendJmsMessage("Hello World!");
|
||||
frame = conn.receiveFrame(2000);
|
||||
assertNull(frame);
|
||||
frame = subscribeQueue(conn, "sub-01", "\\c\\c");
|
||||
assertTrue(frame.getBody().contains("Queue :: does not exist"));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import javax.jms.MessageListener;
|
|||
import javax.jms.MessageProducer;
|
||||
import javax.jms.TextMessage;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
@ -66,7 +67,10 @@ import org.junit.After;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class StompTest extends StompTestBase {
|
||||
|
||||
private static final transient IntegrationTestLogger log = IntegrationTestLogger.LOGGER;
|
||||
|
@ -76,7 +80,7 @@ public class StompTest extends StompTestBase {
|
|||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -94,6 +98,7 @@ public class StompTest extends StompTestBase {
|
|||
}
|
||||
} finally {
|
||||
super.tearDown();
|
||||
conn.closeTransport();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -101,8 +106,10 @@ public class StompTest extends StompTestBase {
|
|||
public void testConnectionTTL() throws Exception {
|
||||
int port = 61614;
|
||||
|
||||
URI uri = createStompClientUri(scheme, hostname, port);
|
||||
|
||||
server.getActiveMQServer().getRemotingService().createAcceptor("test", "tcp://127.0.0.1:" + port + "?connectionTtl=1000").start();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect("brianm", "wombats");
|
||||
|
||||
Thread.sleep(5000);
|
||||
|
@ -257,33 +264,6 @@ public class StompTest extends StompTestBase {
|
|||
clientProvider.disconnect();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSendReceiveLargeMessage() throws Exception {
|
||||
String address = "testLargeMessageAddress";
|
||||
server.getActiveMQServer().createQueue(SimpleString.toSimpleString(address), RoutingType.ANYCAST, SimpleString.toSimpleString(address), null, true, false);
|
||||
|
||||
// STOMP default is UTF-8 == 1 byte per char.
|
||||
int largeMessageStringSize = 10 * 1024 * 1024; // 10MB
|
||||
StringBuilder b = new StringBuilder(largeMessageStringSize);
|
||||
for (int i = 0; i < largeMessageStringSize; i++) {
|
||||
b.append('t');
|
||||
}
|
||||
String payload = b.toString();
|
||||
|
||||
// Set up STOMP subscription
|
||||
conn.connect(defUser, defPass);
|
||||
subscribe(conn, null, Stomp.Headers.Subscribe.AckModeValues.AUTO, null, null, address, true);
|
||||
|
||||
// Send Large Message
|
||||
System.out.println("Sending Message Size: " + largeMessageStringSize);
|
||||
send(conn, address, null, payload);
|
||||
|
||||
// Receive STOMP Message
|
||||
ClientStompFrame frame = conn.receiveFrame();
|
||||
System.out.println(frame.getBody().length());
|
||||
assertTrue(frame.getBody().equals(payload));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void sendMQTTReceiveSTOMP() throws Exception {
|
||||
String payload = "This is a test message";
|
||||
|
@ -936,10 +916,10 @@ public class StompTest extends StompTestBase {
|
|||
if (sendDisconnect) {
|
||||
conn.disconnect();
|
||||
conn.destroy();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
} else {
|
||||
conn.destroy();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
}
|
||||
|
||||
// message should be received since message was not acknowledged
|
||||
|
@ -953,7 +933,7 @@ public class StompTest extends StompTestBase {
|
|||
conn.disconnect();
|
||||
conn.destroy();
|
||||
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
|
||||
// now let's make sure we don't see the message again
|
||||
|
||||
|
@ -1219,7 +1199,7 @@ public class StompTest extends StompTestBase {
|
|||
sendJmsMessage(getName(), topic);
|
||||
|
||||
conn.destroy();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass, "myclientid");
|
||||
|
||||
subscribeTopic(conn, null, null, getName());
|
||||
|
@ -1257,7 +1237,7 @@ public class StompTest extends StompTestBase {
|
|||
assertNotNull(server.getActiveMQServer().locateQueue(SimpleString.toSimpleString("myclientid." + getName())));
|
||||
|
||||
conn.destroy();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
|
||||
conn.connect(defUser, defPass, "myclientid");
|
||||
unsubscribe(conn, getName(), getTopicPrefix() + getTopicName(), false, true);
|
||||
|
@ -1302,7 +1282,7 @@ public class StompTest extends StompTestBase {
|
|||
conn.destroy();
|
||||
|
||||
// connect again
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
// send a receipted message to the topic
|
||||
|
@ -1441,12 +1421,15 @@ public class StompTest extends StompTestBase {
|
|||
|
||||
public void testPrefix(final String prefix, final RoutingType routingType, final boolean send) throws Exception {
|
||||
int port = 61614;
|
||||
|
||||
URI uri = createStompClientUri(scheme, hostname, port);
|
||||
|
||||
final String ADDRESS = UUID.randomUUID().toString();
|
||||
final String PREFIXED_ADDRESS = prefix + ADDRESS;
|
||||
String param = routingType.toString();
|
||||
String urlParam = param.toLowerCase() + "Prefix";
|
||||
server.getActiveMQServer().getRemotingService().createAcceptor("test", "tcp://" + hostname + ":" + port + "?protocols=" + StompProtocolManagerFactory.STOMP_PROTOCOL_NAME + "&" + urlParam + "=" + prefix).start();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
// since this queue doesn't exist the broker should create a new address using the routing type matching the prefix
|
||||
|
@ -1496,11 +1479,14 @@ public class StompTest extends StompTestBase {
|
|||
|
||||
public void testPrefixedSendAndRecieve(final String prefix, RoutingType routingType) throws Exception {
|
||||
int port = 61614;
|
||||
|
||||
URI uri = createStompClientUri(scheme, hostname, port);
|
||||
|
||||
final String ADDRESS = UUID.randomUUID().toString();
|
||||
final String PREFIXED_ADDRESS = prefix + ADDRESS;
|
||||
String urlParam = routingType.toString().toLowerCase() + "Prefix";
|
||||
server.getActiveMQServer().getRemotingService().createAcceptor("test", "tcp://" + hostname + ":" + port + "?protocols=" + StompProtocolManagerFactory.STOMP_PROTOCOL_NAME + "&" + urlParam + "=" + prefix).start();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
String uuid = UUID.randomUUID().toString();
|
||||
|
||||
|
|
|
@ -26,7 +26,11 @@ import javax.jms.Session;
|
|||
import javax.jms.TextMessage;
|
||||
import javax.jms.Topic;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -62,9 +66,22 @@ import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConne
|
|||
import org.apache.activemq.artemis.tests.unit.util.InVMNamingContext;
|
||||
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
|
||||
import org.junit.Before;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public abstract class StompTestBase extends ActiveMQTestBase {
|
||||
|
||||
@Parameterized.Parameter
|
||||
public String scheme;
|
||||
|
||||
protected URI uri;
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> data() {
|
||||
return Arrays.asList(new Object[][]{{"ws+v10.stomp"}, {"tcp+v10.stomp"}});
|
||||
}
|
||||
|
||||
protected String hostname = "127.0.0.1";
|
||||
|
||||
protected final int port = 61613;
|
||||
|
@ -120,8 +137,13 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
|
||||
uri = new URI(scheme + "://" + hostname + ":" + port);
|
||||
|
||||
server = createServer();
|
||||
server.start();
|
||||
|
||||
waitForServerToStart(server.getActiveMQServer());
|
||||
|
||||
connectionFactory = createConnectionFactory();
|
||||
|
||||
((ActiveMQConnectionFactory)connectionFactory).setCompressLargeMessage(isCompressLargeMessages());
|
||||
|
@ -330,7 +352,7 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
String subscriptionId,
|
||||
String ack,
|
||||
String durableId) throws IOException, InterruptedException {
|
||||
return subscribe(conn, subscriptionId, ack, durableId, false);
|
||||
return subscribe(conn, subscriptionId, ack, durableId, true);
|
||||
}
|
||||
|
||||
public ClientStompFrame subscribe(StompClientConnection conn,
|
||||
|
@ -346,7 +368,7 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
String ack,
|
||||
String durableId,
|
||||
String selector) throws IOException, InterruptedException {
|
||||
return subscribe(conn, subscriptionId, ack, durableId, selector, false);
|
||||
return subscribe(conn, subscriptionId, ack, durableId, selector, true);
|
||||
}
|
||||
|
||||
public ClientStompFrame subscribe(StompClientConnection conn,
|
||||
|
@ -358,8 +380,8 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
return subscribe(conn, subscriptionId, ack, durableId, selector, getQueuePrefix() + getQueueName(), receipt);
|
||||
}
|
||||
|
||||
public void subscribeQueue(StompClientConnection conn, String subId, String destination) throws IOException, InterruptedException {
|
||||
subscribe(conn, subId, Stomp.Headers.Subscribe.AckModeValues.AUTO, null, null, destination, false);
|
||||
public ClientStompFrame subscribeQueue(StompClientConnection conn, String subId, String destination) throws IOException, InterruptedException {
|
||||
return subscribe(conn, subId, Stomp.Headers.Subscribe.AckModeValues.AUTO, null, null, destination, true);
|
||||
}
|
||||
|
||||
public ClientStompFrame subscribe(StompClientConnection conn,
|
||||
|
@ -384,6 +406,7 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
if (selector != null) {
|
||||
frame.addHeader(Stomp.Headers.Subscribe.SELECTOR, selector);
|
||||
}
|
||||
|
||||
String uuid = UUID.randomUUID().toString();
|
||||
if (receipt) {
|
||||
frame.addHeader(Stomp.Headers.RECEIPT_REQUESTED, uuid);
|
||||
|
@ -391,6 +414,11 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
|
||||
frame = conn.sendFrame(frame);
|
||||
|
||||
// Return Error Frame back to the client
|
||||
if (frame != null && frame.getCommand().equals("ERROR")) {
|
||||
return frame;
|
||||
}
|
||||
|
||||
if (receipt) {
|
||||
assertEquals(uuid, frame.getHeader(Stomp.Headers.Response.RECEIPT_ID));
|
||||
}
|
||||
|
@ -402,7 +430,7 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
String subscriptionId,
|
||||
String ack,
|
||||
String durableId) throws IOException, InterruptedException {
|
||||
return subscribeTopic(conn, subscriptionId, ack, durableId, false);
|
||||
return subscribeTopic(conn, subscriptionId, ack, durableId, true);
|
||||
}
|
||||
|
||||
public ClientStompFrame subscribeTopic(StompClientConnection conn,
|
||||
|
@ -441,6 +469,10 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
|
||||
frame = conn.sendFrame(frame);
|
||||
|
||||
if (frame.getCommand().equals("ERROR")) {
|
||||
return frame;
|
||||
}
|
||||
|
||||
if (receipt) {
|
||||
assertNotNull("Requested receipt, but response is null", frame);
|
||||
assertTrue(frame.getHeader(Stomp.Headers.Response.RECEIPT_ID).equals(uuid));
|
||||
|
@ -527,6 +559,10 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
}
|
||||
frame = conn.sendFrame(frame);
|
||||
|
||||
if (frame != null && frame.getCommand().equals("ERROR")) {
|
||||
return frame;
|
||||
}
|
||||
|
||||
if (receipt) {
|
||||
assertEquals(Stomp.Responses.RECEIPT, frame.getCommand());
|
||||
assertEquals(uuid, frame.getHeader(Stomp.Headers.Response.RECEIPT_ID));
|
||||
|
@ -536,4 +572,8 @@ public abstract class StompTestBase extends ActiveMQTestBase {
|
|||
|
||||
return frame;
|
||||
}
|
||||
|
||||
public URI createStompClientUri(String scheme, String hostname, int port) throws URISyntaxException {
|
||||
return new URI(scheme + "://" + hostname + ":" + port);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,13 +24,21 @@ import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConne
|
|||
import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConnectionFactory;
|
||||
import org.apache.felix.resolver.util.ArrayMap;
|
||||
import org.junit.Test;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class StompTestPropertiesInterceptor extends StompTestBase {
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> data() {
|
||||
return Arrays.asList(new Object[][]{{"ws+v12.stomp"}, {"tcp+v12.stomp"}});
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getIncomingInterceptors() {
|
||||
List<String> stompIncomingInterceptor = new ArrayList<>();
|
||||
|
@ -73,7 +81,7 @@ public class StompTestPropertiesInterceptor extends StompTestBase {
|
|||
expectedProperties.put(MESSAGE_TEXT, msgText);
|
||||
expectedProperties.put(MY_HEADER, myHeader);
|
||||
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
ClientStompFrame subFrame = conn.createFrame("SUBSCRIBE");
|
||||
|
|
|
@ -62,7 +62,7 @@ public class StompTestWithInterceptors extends StompTestBase {
|
|||
// So we clear them here
|
||||
MyCoreInterceptor.incomingInterceptedFrames.clear();
|
||||
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
ClientStompFrame subFrame = conn.createFrame("SUBSCRIBE");
|
||||
|
|
|
@ -16,19 +16,34 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.tests.integration.stomp;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.api.core.client.ActiveMQClient;
|
||||
import org.apache.activemq.artemis.tests.integration.IntegrationTestLogger;
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.Stomp;
|
||||
import org.apache.activemq.artemis.tests.integration.largemessage.LargeMessageTestBase;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.ClientStompFrame;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConnection;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConnectionFactory;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
@Ignore
|
||||
public class StompTestWithLargeMessages extends StompTestBase {
|
||||
|
||||
// Web Socket has max frame size of 64kb. Large message tests only available over TCP.
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> data() {
|
||||
return Arrays.asList(new Object[][]{{"tcp+v10.stomp"}, {"tcp+v12.stomp"}});
|
||||
}
|
||||
|
||||
@Override
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
@ -50,10 +65,39 @@ public class StompTestWithLargeMessages extends StompTestBase {
|
|||
return 2048;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSendReceiveLargeMessage() throws Exception {
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
|
||||
String address = "testLargeMessageAddress";
|
||||
server.getActiveMQServer().createQueue(SimpleString.toSimpleString(address), RoutingType.ANYCAST, SimpleString.toSimpleString(address), null, true, false);
|
||||
|
||||
// STOMP default is UTF-8 == 1 byte per char.
|
||||
int largeMessageStringSize = 10 * 1024 * 1024; // 10MB
|
||||
StringBuilder b = new StringBuilder(largeMessageStringSize);
|
||||
for (int i = 0; i < largeMessageStringSize; i++) {
|
||||
b.append('t');
|
||||
}
|
||||
String payload = b.toString();
|
||||
|
||||
// Set up STOMP subscription
|
||||
conn.connect(defUser, defPass);
|
||||
subscribe(conn, null, Stomp.Headers.Subscribe.AckModeValues.AUTO, null, null, address, true);
|
||||
|
||||
// Send Large Message
|
||||
System.out.println("Sending Message Size: " + largeMessageStringSize);
|
||||
send(conn, address, null, payload);
|
||||
|
||||
// Receive STOMP Message
|
||||
ClientStompFrame frame = conn.receiveFrame();
|
||||
System.out.println(frame.getBody().length());
|
||||
assertTrue(frame.getBody().equals(payload));
|
||||
}
|
||||
|
||||
//stomp sender -> large -> stomp receiver
|
||||
@Test
|
||||
public void testSendReceiveLargePersistentMessages() throws Exception {
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
int count = 10;
|
||||
|
@ -101,7 +145,7 @@ public class StompTestWithLargeMessages extends StompTestBase {
|
|||
//core sender -> large -> stomp receiver
|
||||
@Test
|
||||
public void testReceiveLargePersistentMessagesFromCore() throws Exception {
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
int msgSize = 3 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE;
|
||||
|
@ -142,103 +186,103 @@ public class StompTestWithLargeMessages extends StompTestBase {
|
|||
conn.disconnect();
|
||||
}
|
||||
|
||||
//stomp v12 sender -> large -> stomp v12 receiver
|
||||
@Test
|
||||
public void testSendReceiveLargePersistentMessagesV12() throws Exception {
|
||||
StompClientConnection connV12 = StompClientConnectionFactory.createClientConnection("1.2", "localhost", port);
|
||||
connV12.connect(defUser, defPass);
|
||||
|
||||
int count = 10;
|
||||
int szBody = 1024 * 1024;
|
||||
char[] contents = new char[szBody];
|
||||
for (int i = 0; i < szBody; i++) {
|
||||
contents[i] = 'A';
|
||||
}
|
||||
String body = new String(contents);
|
||||
|
||||
ClientStompFrame frame = connV12.createFrame("SEND");
|
||||
frame.addHeader("destination-type", "ANYCAST");
|
||||
frame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
frame.addHeader("persistent", "true");
|
||||
frame.setBody(body);
|
||||
|
||||
for (int i = 0; i < count; i++) {
|
||||
connV12.sendFrame(frame);
|
||||
}
|
||||
|
||||
ClientStompFrame subFrame = connV12.createFrame("SUBSCRIBE");
|
||||
subFrame.addHeader("id", "a-sub");
|
||||
subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
subFrame.addHeader("ack", "auto");
|
||||
|
||||
connV12.sendFrame(subFrame);
|
||||
|
||||
for (int i = 0; i < count; i++) {
|
||||
ClientStompFrame receiveFrame = connV12.receiveFrame(30000);
|
||||
|
||||
Assert.assertNotNull(receiveFrame);
|
||||
System.out.println("part of frame: " + receiveFrame.getBody().substring(0, 20));
|
||||
Assert.assertTrue(receiveFrame.getCommand().equals("MESSAGE"));
|
||||
Assert.assertEquals(receiveFrame.getHeader("destination"), getQueuePrefix() + getQueueName());
|
||||
assertEquals(szBody, receiveFrame.getBody().length());
|
||||
}
|
||||
|
||||
// remove susbcription
|
||||
ClientStompFrame unsubFrame = connV12.createFrame("UNSUBSCRIBE");
|
||||
unsubFrame.addHeader("id", "a-sub");
|
||||
connV12.sendFrame(unsubFrame);
|
||||
|
||||
connV12.disconnect();
|
||||
}
|
||||
|
||||
//core sender -> large -> stomp v12 receiver
|
||||
@Test
|
||||
public void testReceiveLargePersistentMessagesFromCoreV12() throws Exception {
|
||||
int msgSize = 3 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE;
|
||||
char[] contents = new char[msgSize];
|
||||
for (int i = 0; i < msgSize; i++) {
|
||||
contents[i] = 'B';
|
||||
}
|
||||
String msg = new String(contents);
|
||||
|
||||
int count = 10;
|
||||
for (int i = 0; i < count; i++) {
|
||||
this.sendJmsMessage(msg);
|
||||
}
|
||||
|
||||
StompClientConnection connV12 = StompClientConnectionFactory.createClientConnection("1.2", "localhost", port);
|
||||
connV12.connect(defUser, defPass);
|
||||
|
||||
ClientStompFrame subFrame = connV12.createFrame("SUBSCRIBE");
|
||||
subFrame.addHeader("id", "a-sub");
|
||||
subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
subFrame.addHeader("ack", "auto");
|
||||
connV12.sendFrame(subFrame);
|
||||
|
||||
for (int i = 0; i < count; i++) {
|
||||
ClientStompFrame receiveFrame = connV12.receiveFrame(30000);
|
||||
|
||||
Assert.assertNotNull(receiveFrame);
|
||||
System.out.println("part of frame: " + receiveFrame.getBody().substring(0, 20));
|
||||
Assert.assertTrue(receiveFrame.getCommand().equals("MESSAGE"));
|
||||
Assert.assertEquals(receiveFrame.getHeader("destination"), getQueuePrefix() + getQueueName());
|
||||
assertEquals(msgSize, receiveFrame.getBody().length());
|
||||
}
|
||||
|
||||
// remove susbcription
|
||||
ClientStompFrame unsubFrame = connV12.createFrame("UNSUBSCRIBE");
|
||||
unsubFrame.addHeader("id", "a-sub");
|
||||
connV12.sendFrame(unsubFrame);
|
||||
|
||||
connV12.disconnect();
|
||||
}
|
||||
// //stomp v12 sender -> large -> stomp v12 receiver
|
||||
// @Test
|
||||
// public void testSendReceiveLargePersistentMessagesV12() throws Exception {
|
||||
// StompClientConnection connV12 = StompClientConnectionFactory.createClientConnection("1.2", "localhost", port);
|
||||
// connV12.connect(defUser, defPass);
|
||||
//
|
||||
// int count = 10;
|
||||
// int szBody = 1024 * 1024;
|
||||
// char[] contents = new char[szBody];
|
||||
// for (int i = 0; i < szBody; i++) {
|
||||
// contents[i] = 'A';
|
||||
// }
|
||||
// String body = new String(contents);
|
||||
//
|
||||
// ClientStompFrame frame = connV12.createFrame("SEND");
|
||||
// frame.addHeader("destination-type", "ANYCAST");
|
||||
// frame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
// frame.addHeader("persistent", "true");
|
||||
// frame.setBody(body);
|
||||
//
|
||||
// for (int i = 0; i < count; i++) {
|
||||
// connV12.sendFrame(frame);
|
||||
// }
|
||||
//
|
||||
// ClientStompFrame subFrame = connV12.createFrame("SUBSCRIBE");
|
||||
// subFrame.addHeader("id", "a-sub");
|
||||
// subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
// subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
// subFrame.addHeader("ack", "auto");
|
||||
//
|
||||
// connV12.sendFrame(subFrame);
|
||||
//
|
||||
// for (int i = 0; i < count; i++) {
|
||||
// ClientStompFrame receiveFrame = connV12.receiveFrame(30000);
|
||||
//
|
||||
// Assert.assertNotNull(receiveFrame);
|
||||
// System.out.println("part of frame: " + receiveFrame.getBody().substring(0, 20));
|
||||
// Assert.assertTrue(receiveFrame.getCommand().equals("MESSAGE"));
|
||||
// Assert.assertEquals(receiveFrame.getHeader("destination"), getQueuePrefix() + getQueueName());
|
||||
// assertEquals(szBody, receiveFrame.getBody().length());
|
||||
// }
|
||||
//
|
||||
// // remove susbcription
|
||||
// ClientStompFrame unsubFrame = connV12.createFrame("UNSUBSCRIBE");
|
||||
// unsubFrame.addHeader("id", "a-sub");
|
||||
// connV12.sendFrame(unsubFrame);
|
||||
//
|
||||
// connV12.disconnect();
|
||||
// }
|
||||
//
|
||||
// //core sender -> large -> stomp v12 receiver
|
||||
// @Test
|
||||
// public void testReceiveLargePersistentMessagesFromCoreV12() throws Exception {
|
||||
// int msgSize = 3 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE;
|
||||
// char[] contents = new char[msgSize];
|
||||
// for (int i = 0; i < msgSize; i++) {
|
||||
// contents[i] = 'B';
|
||||
// }
|
||||
// String msg = new String(contents);
|
||||
//
|
||||
// int count = 10;
|
||||
// for (int i = 0; i < count; i++) {
|
||||
// this.sendJmsMessage(msg);
|
||||
// }
|
||||
//
|
||||
// StompClientConnection connV12 = StompClientConnectionFactory.createClientConnection("1.2", "localhost", port);
|
||||
// connV12.connect(defUser, defPass);
|
||||
//
|
||||
// ClientStompFrame subFrame = connV12.createFrame("SUBSCRIBE");
|
||||
// subFrame.addHeader("id", "a-sub");
|
||||
// subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
// subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
// subFrame.addHeader("ack", "auto");
|
||||
// connV12.sendFrame(subFrame);
|
||||
//
|
||||
// for (int i = 0; i < count; i++) {
|
||||
// ClientStompFrame receiveFrame = connV12.receiveFrame(30000);
|
||||
//
|
||||
// Assert.assertNotNull(receiveFrame);
|
||||
// System.out.println("part of frame: " + receiveFrame.getBody().substring(0, 20));
|
||||
// Assert.assertTrue(receiveFrame.getCommand().equals("MESSAGE"));
|
||||
// Assert.assertEquals(receiveFrame.getHeader("destination"), getQueuePrefix() + getQueueName());
|
||||
// assertEquals(msgSize, receiveFrame.getBody().length());
|
||||
// }
|
||||
//
|
||||
// // remove susbcription
|
||||
// ClientStompFrame unsubFrame = connV12.createFrame("UNSUBSCRIBE");
|
||||
// unsubFrame.addHeader("id", "a-sub");
|
||||
// connV12.sendFrame(unsubFrame);
|
||||
//
|
||||
// connV12.disconnect();
|
||||
// }
|
||||
|
||||
//core sender -> large (compressed regular) -> stomp v10 receiver
|
||||
@Test
|
||||
public void testReceiveLargeCompressedToRegularPersistentMessagesFromCore() throws Exception {
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
LargeMessageTestBase.TestLargeMessageInputStream input = new LargeMessageTestBase.TestLargeMessageInputStream(ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, true);
|
||||
|
@ -281,136 +325,142 @@ public class StompTestWithLargeMessages extends StompTestBase {
|
|||
conn.disconnect();
|
||||
}
|
||||
|
||||
//core sender -> large (compressed regular) -> stomp v12 receiver
|
||||
@Test
|
||||
public void testReceiveLargeCompressedToRegularPersistentMessagesFromCoreV12() throws Exception {
|
||||
LargeMessageTestBase.TestLargeMessageInputStream input = new LargeMessageTestBase.TestLargeMessageInputStream(ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, true);
|
||||
LargeMessageTestBase.adjustLargeCompression(true, input, ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
|
||||
char[] contents = input.toArray();
|
||||
String msg = new String(contents);
|
||||
|
||||
int count = 10;
|
||||
for (int i = 0; i < count; i++) {
|
||||
this.sendJmsMessage(msg);
|
||||
}
|
||||
|
||||
StompClientConnection connV12 = StompClientConnectionFactory.createClientConnection("1.2", "localhost", port);
|
||||
connV12.connect(defUser, defPass);
|
||||
|
||||
ClientStompFrame subFrame = connV12.createFrame("SUBSCRIBE");
|
||||
subFrame.addHeader("id", "a-sub");
|
||||
subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
subFrame.addHeader("ack", "auto");
|
||||
|
||||
connV12.sendFrame(subFrame);
|
||||
|
||||
for (int i = 0; i < count; i++) {
|
||||
ClientStompFrame receiveFrame = connV12.receiveFrame(30000);
|
||||
|
||||
Assert.assertNotNull(receiveFrame);
|
||||
System.out.println("part of frame: " + receiveFrame.getBody().substring(0, 20));
|
||||
Assert.assertTrue(receiveFrame.getCommand().equals("MESSAGE"));
|
||||
Assert.assertEquals(receiveFrame.getHeader("destination"), getQueuePrefix() + getQueueName());
|
||||
assertEquals(contents.length, receiveFrame.getBody().length());
|
||||
}
|
||||
|
||||
// remove susbcription
|
||||
ClientStompFrame unsubFrame = connV12.createFrame("UNSUBSCRIBE");
|
||||
unsubFrame.addHeader("id", "a-sub");
|
||||
connV12.sendFrame(unsubFrame);
|
||||
|
||||
connV12.disconnect();
|
||||
}
|
||||
|
||||
//core sender -> large (compressed large) -> stomp v12 receiver
|
||||
@Test
|
||||
public void testReceiveLargeCompressedToLargePersistentMessagesFromCoreV12() throws Exception {
|
||||
LargeMessageTestBase.TestLargeMessageInputStream input = new LargeMessageTestBase.TestLargeMessageInputStream(ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, true);
|
||||
input.setSize(10 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
LargeMessageTestBase.adjustLargeCompression(false, input, 10 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
|
||||
char[] contents = input.toArray();
|
||||
String msg = new String(contents);
|
||||
|
||||
int count = 10;
|
||||
for (int i = 0; i < count; i++) {
|
||||
this.sendJmsMessage(msg);
|
||||
}
|
||||
|
||||
IntegrationTestLogger.LOGGER.info("Message count for " + getQueueName() + ": " + server.getActiveMQServer().locateQueue(SimpleString.toSimpleString(getQueueName())).getMessageCount());
|
||||
|
||||
StompClientConnection connV12 = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
connV12.connect(defUser, defPass);
|
||||
|
||||
ClientStompFrame subFrame = connV12.createFrame("SUBSCRIBE");
|
||||
subFrame.addHeader("id", "a-sub");
|
||||
subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
subFrame.addHeader("ack", "auto");
|
||||
|
||||
connV12.sendFrame(subFrame);
|
||||
|
||||
for (int i = 0; i < count; i++) {
|
||||
ClientStompFrame receiveFrame = connV12.receiveFrame(30000);
|
||||
|
||||
Assert.assertNotNull(receiveFrame);
|
||||
System.out.println("part of frame: " + receiveFrame.getBody().substring(0, 20));
|
||||
Assert.assertTrue(receiveFrame.getCommand().equals("MESSAGE"));
|
||||
Assert.assertEquals(receiveFrame.getHeader("destination"), getQueuePrefix() + getQueueName());
|
||||
assertEquals(contents.length, receiveFrame.getBody().length());
|
||||
}
|
||||
|
||||
// remove susbcription
|
||||
ClientStompFrame unsubFrame = connV12.createFrame("UNSUBSCRIBE");
|
||||
unsubFrame.addHeader("id", "a-sub");
|
||||
connV12.sendFrame(unsubFrame);
|
||||
|
||||
connV12.disconnect();
|
||||
}
|
||||
// //core sender -> large (compressed regular) -> stomp v12 receiver
|
||||
// @Test
|
||||
// public void testReceiveLargeCompressedToRegularPersistentMessagesFromCoreV12() throws Exception {
|
||||
// LargeMessageTestBase.TestLargeMessageInputStream input = new LargeMessageTestBase.TestLargeMessageInputStream(ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, true);
|
||||
// LargeMessageTestBase.adjustLargeCompression(true, input, ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
//
|
||||
// char[] contents = input.toArray();
|
||||
// String msg = new String(contents);
|
||||
//
|
||||
// int count = 10;
|
||||
// for (int i = 0; i < count; i++) {
|
||||
// this.sendJmsMessage(msg);
|
||||
// }
|
||||
//
|
||||
// StompClientConnection connV12 = StompClientConnectionFactory.createClientConnection("1.2", "localhost", port);
|
||||
// connV12.connect(defUser, defPass);
|
||||
//
|
||||
// ClientStompFrame subFrame = connV12.createFrame("SUBSCRIBE");
|
||||
// subFrame.addHeader("id", "a-sub");
|
||||
// subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
// subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
// subFrame.addHeader("ack", "auto");
|
||||
//
|
||||
// connV12.sendFrame(subFrame);
|
||||
//
|
||||
// for (int i = 0; i < count; i++) {
|
||||
// ClientStompFrame receiveFrame = connV12.receiveFrame(30000);
|
||||
//
|
||||
// Assert.assertNotNull(receiveFrame);
|
||||
// System.out.println("part of frame: " + receiveFrame.getBody().substring(0, 20));
|
||||
// Assert.assertTrue(receiveFrame.getCommand().equals("MESSAGE"));
|
||||
// Assert.assertEquals(receiveFrame.getHeader("destination"), getQueuePrefix() + getQueueName());
|
||||
// assertEquals(contents.length, receiveFrame.getBody().length());
|
||||
// }
|
||||
//
|
||||
// // remove susbcription
|
||||
// ClientStompFrame unsubFrame = connV12.createFrame("UNSUBSCRIBE");
|
||||
// unsubFrame.addHeader("id", "a-sub");
|
||||
// connV12.sendFrame(unsubFrame);
|
||||
//
|
||||
// connV12.disconnect();
|
||||
// }
|
||||
//
|
||||
// //core sender -> large (compressed large) -> stomp v12 receiver
|
||||
// @Test
|
||||
// public void testReceiveLargeCompressedToLargePersistentMessagesFromCoreV12() throws Exception {
|
||||
// LargeMessageTestBase.TestLargeMessageInputStream input = new LargeMessageTestBase.TestLargeMessageInputStream(ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, true);
|
||||
// input.setSize(10 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
// LargeMessageTestBase.adjustLargeCompression(false, input, 10 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
//
|
||||
// char[] contents = input.toArray();
|
||||
// String msg = new String(contents);
|
||||
//
|
||||
// int count = 10;
|
||||
// for (int i = 0; i < count; i++) {
|
||||
// this.sendJmsMessage(msg);
|
||||
// }
|
||||
//
|
||||
// IntegrationTestLogger.LOGGER.info("Message count for " + getQueueName() + ": " + server.getActiveMQServer().locateQueue(SimpleString.toSimpleString(getQueueName())).getMessageCount());
|
||||
//
|
||||
// StompClientConnection connV12 = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
// connV12.connect(defUser, defPass);
|
||||
//
|
||||
// ClientStompFrame subFrame = connV12.createFrame("SUBSCRIBE");
|
||||
// subFrame.addHeader("id", "a-sub");
|
||||
// subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
// subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
// subFrame.addHeader("ack", "auto");
|
||||
//
|
||||
// connV12.sendFrame(subFrame);
|
||||
//
|
||||
// for (int i = 0; i < count; i++) {
|
||||
// ClientStompFrame receiveFrame = connV12.receiveFrame(30000);
|
||||
//
|
||||
// Assert.assertNotNull(receiveFrame);
|
||||
// System.out.println("part of frame: " + receiveFrame.getBody().substring(0, 20));
|
||||
// Assert.assertTrue(receiveFrame.getCommand().equals("MESSAGE"));
|
||||
// Assert.assertEquals(receiveFrame.getHeader("destination"), getQueuePrefix() + getQueueName());
|
||||
// assertEquals(contents.length, receiveFrame.getBody().length());
|
||||
// }
|
||||
//
|
||||
// // remove susbcription
|
||||
// ClientStompFrame unsubFrame = connV12.createFrame("UNSUBSCRIBE");
|
||||
// unsubFrame.addHeader("id", "a-sub");
|
||||
// connV12.sendFrame(unsubFrame);
|
||||
//
|
||||
// connV12.disconnect();
|
||||
// }
|
||||
|
||||
//core sender -> large (compressed large) -> stomp v10 receiver
|
||||
@Test
|
||||
public void testReceiveLargeCompressedToLargePersistentMessagesFromCore() throws Exception {
|
||||
LargeMessageTestBase.TestLargeMessageInputStream input = new LargeMessageTestBase.TestLargeMessageInputStream(ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, true);
|
||||
input.setSize(10 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
LargeMessageTestBase.adjustLargeCompression(false, input, 10 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
|
||||
char[] contents = input.toArray();
|
||||
String msg = new String(contents);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
try {
|
||||
LargeMessageTestBase.TestLargeMessageInputStream input = new LargeMessageTestBase.TestLargeMessageInputStream(ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE, true);
|
||||
input.setSize(10 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
LargeMessageTestBase.adjustLargeCompression(false, input, 10 * ActiveMQClient.DEFAULT_MIN_LARGE_MESSAGE_SIZE);
|
||||
|
||||
String leadingPart = msg.substring(0, 100);
|
||||
char[] contents = input.toArray();
|
||||
String msg = new String(contents);
|
||||
|
||||
int count = 10;
|
||||
for (int i = 0; i < count; i++) {
|
||||
this.sendJmsMessage(msg);
|
||||
String leadingPart = msg.substring(0, 100);
|
||||
|
||||
int count = 10;
|
||||
for (int i = 0; i < count; i++) {
|
||||
this.sendJmsMessage(msg);
|
||||
}
|
||||
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
ClientStompFrame subFrame = conn.createFrame("SUBSCRIBE");
|
||||
subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
subFrame.addHeader("ack", "auto");
|
||||
conn.sendFrame(subFrame);
|
||||
|
||||
for (int i = 0; i < count; i++) {
|
||||
ClientStompFrame frame = conn.receiveFrame(60000);
|
||||
Assert.assertNotNull(frame);
|
||||
System.out.println(frame.toString());
|
||||
System.out.println("part of frame: " + frame.getBody().substring(0, 250));
|
||||
Assert.assertTrue(frame.getCommand().equals("MESSAGE"));
|
||||
Assert.assertTrue(frame.getHeader("destination").equals(getQueuePrefix() + getQueueName()));
|
||||
int index = frame.getBody().toString().indexOf(leadingPart);
|
||||
assertEquals(msg.length(), (frame.getBody().toString().length() - index));
|
||||
}
|
||||
|
||||
ClientStompFrame unsubFrame = conn.createFrame("UNSUBSCRIBE");
|
||||
unsubFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
unsubFrame.addHeader("receipt", "567");
|
||||
conn.sendFrame(unsubFrame);
|
||||
} finally {
|
||||
conn.disconnect();
|
||||
conn.closeTransport();
|
||||
}
|
||||
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
ClientStompFrame subFrame = conn.createFrame("SUBSCRIBE");
|
||||
subFrame.addHeader("subscription-type", "ANYCAST");
|
||||
subFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
subFrame.addHeader("ack", "auto");
|
||||
conn.sendFrame(subFrame);
|
||||
|
||||
for (int i = 0; i < count; i++) {
|
||||
ClientStompFrame frame = conn.receiveFrame(60000);
|
||||
Assert.assertNotNull(frame);
|
||||
System.out.println("part of frame: " + frame.getBody().substring(0, 250));
|
||||
Assert.assertTrue(frame.getCommand().equals("MESSAGE"));
|
||||
Assert.assertTrue(frame.getHeader("destination").equals(getQueuePrefix() + getQueueName()));
|
||||
int index = frame.getBody().toString().indexOf(leadingPart);
|
||||
assertEquals(msg.length(), (frame.getBody().toString().length() - index));
|
||||
}
|
||||
|
||||
ClientStompFrame unsubFrame = conn.createFrame("UNSUBSCRIBE");
|
||||
unsubFrame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
unsubFrame.addHeader("receipt", "567");
|
||||
conn.sendFrame(unsubFrame);
|
||||
|
||||
conn.disconnect();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ public class StompTestWithMessageID extends StompTestBase {
|
|||
|
||||
@Test
|
||||
public void testEnableMessageID() throws Exception {
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
ClientStompFrame frame = conn.createFrame("SEND");
|
||||
|
@ -74,5 +74,7 @@ public class StompTestWithMessageID extends StompTestBase {
|
|||
|
||||
message = (TextMessage) consumer.receive(2000);
|
||||
Assert.assertNull(message);
|
||||
|
||||
conn.disconnect();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ public class StompTestWithSecurity extends StompTestBase {
|
|||
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
ClientStompFrame frame = conn.createFrame("SEND");
|
||||
|
|
|
@ -0,0 +1,94 @@
|
|||
/*
|
||||
* 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.artemis.tests.integration.stomp;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.activemq.artemis.junit.Wait;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.ClientStompFrame;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConnection;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConnectionFactory;
|
||||
import org.apache.commons.lang.RandomStringUtils;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class StompWebSocketMaxFrameTest extends StompTestBase {
|
||||
|
||||
private URI wsURI;
|
||||
|
||||
private int wsport = 61614;
|
||||
|
||||
private int stompWSMaxFrameSize = 131072; // 128kb
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> data() {
|
||||
return Arrays.asList(new Object[][]{{"ws+v10.stomp"}, {"ws+v11.stomp"}, {"ws+v12.stomp"}});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
server.getActiveMQServer().getRemotingService().createAcceptor("test", "tcp://127.0.0.1:" + wsport + "?stompMaxFramePayloadLength=" + stompWSMaxFrameSize).start();
|
||||
wsURI = createStompClientUri(scheme, hostname, wsport);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStompSendReceiveWithMaxFramePayloadLength() throws Exception {
|
||||
// Assert that sending message > default 64kb fails
|
||||
int size = 65536;
|
||||
String largeString1 = RandomStringUtils.randomAlphabetic(size);
|
||||
String largeString2 = RandomStringUtils.randomAlphabetic(size);
|
||||
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri, false);
|
||||
conn.getTransport().setMaxFrameSize(stompWSMaxFrameSize);
|
||||
conn.getTransport().connect();
|
||||
|
||||
StompClientConnection conn2 = StompClientConnectionFactory.createClientConnection(wsURI, false);
|
||||
conn2.getTransport().setMaxFrameSize(stompWSMaxFrameSize);
|
||||
conn2.getTransport().connect();
|
||||
|
||||
Wait.waitFor(() -> conn2.getTransport().isConnected() && conn.getTransport().isConnected(), 10000);
|
||||
conn.connect();
|
||||
conn2.connect();
|
||||
|
||||
subscribeQueue(conn2, "sub1", getQueuePrefix() + getQueueName());
|
||||
|
||||
try {
|
||||
// Client is kicked when sending frame > largest frame size.
|
||||
send(conn, getQueuePrefix() + getQueueName(), "text/plain", largeString1, false);
|
||||
Wait.waitFor(() -> !conn.getTransport().isConnected(), 2000);
|
||||
assertFalse(conn.getTransport().isConnected());
|
||||
|
||||
send(conn2, getQueuePrefix() + getQueueName(), "text/plain", largeString2, false);
|
||||
Wait.waitFor(() -> !conn2.getTransport().isConnected(), 2000);
|
||||
assertTrue(conn2.getTransport().isConnected());
|
||||
|
||||
ClientStompFrame frame = conn2.receiveFrame();
|
||||
assertNotNull(frame);
|
||||
assertEquals(largeString2, frame.getBody());
|
||||
|
||||
} finally {
|
||||
conn2.closeTransport();
|
||||
conn.closeTransport();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -24,6 +24,8 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.Stomp;
|
||||
|
||||
public abstract class AbstractClientStompFrame implements ClientStompFrame {
|
||||
|
@ -88,6 +90,16 @@ public abstract class AbstractClientStompFrame implements ClientStompFrame {
|
|||
return toByteBufferInternal(str);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuf toNettyByteBuf() {
|
||||
return Unpooled.copiedBuffer(toByteBuffer());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuf toNettyByteBufWithExtras(String str) {
|
||||
return Unpooled.copiedBuffer(toByteBufferWithExtra(str));
|
||||
}
|
||||
|
||||
public ByteBuffer toByteBufferInternal(String str) {
|
||||
StringBuffer sb = new StringBuffer();
|
||||
sb.append(command + EOL);
|
||||
|
|
|
@ -17,9 +17,8 @@
|
|||
package org.apache.activemq.artemis.tests.integration.stomp.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.URI;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
@ -27,8 +26,15 @@ import java.util.concurrent.BlockingQueue;
|
|||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.channel.ChannelFuture;
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.Stomp;
|
||||
import org.apache.activemq.artemis.tests.integration.IntegrationTestLogger;
|
||||
import org.apache.activemq.artemis.tests.util.Wait;
|
||||
import org.apache.activemq.transport.netty.NettyTransport;
|
||||
import org.apache.activemq.transport.netty.NettyTransportFactory;
|
||||
import org.apache.activemq.transport.netty.NettyTransportListener;
|
||||
|
||||
public abstract class AbstractStompClientConnection implements StompClientConnection {
|
||||
|
||||
|
@ -39,41 +45,69 @@ public abstract class AbstractStompClientConnection implements StompClientConnec
|
|||
protected String username;
|
||||
protected String passcode;
|
||||
protected StompFrameFactory factory;
|
||||
protected final SocketChannel socketChannel;
|
||||
protected NettyTransport transport;
|
||||
protected ByteBuffer readBuffer;
|
||||
protected List<Byte> receiveList;
|
||||
protected BlockingQueue<ClientStompFrame> frameQueue = new LinkedBlockingQueue<>();
|
||||
protected boolean connected = false;
|
||||
protected int serverPingCounter;
|
||||
protected ReaderThread readerThread;
|
||||
//protected ReaderThread readerThread;
|
||||
protected String scheme;
|
||||
|
||||
@Deprecated
|
||||
public AbstractStompClientConnection(String version, String host, int port) throws IOException {
|
||||
this.version = version;
|
||||
this.host = host;
|
||||
this.port = port;
|
||||
this.scheme = "tcp";
|
||||
|
||||
this.factory = StompFrameFactoryFactory.getFactory(version);
|
||||
socketChannel = SocketChannel.open();
|
||||
initSocket();
|
||||
}
|
||||
|
||||
private void initSocket() throws IOException {
|
||||
socketChannel.configureBlocking(true);
|
||||
InetSocketAddress remoteAddr = new InetSocketAddress(host, port);
|
||||
socketChannel.connect(remoteAddr);
|
||||
public AbstractStompClientConnection(URI uri) throws Exception {
|
||||
parseURI(uri);
|
||||
this.factory = StompFrameFactoryFactory.getFactory(version);
|
||||
|
||||
startReaderThread();
|
||||
}
|
||||
|
||||
private void startReaderThread() {
|
||||
readBuffer = ByteBuffer.allocateDirect(10240);
|
||||
receiveList = new ArrayList<>(10240);
|
||||
|
||||
readerThread = new ReaderThread();
|
||||
readerThread.start();
|
||||
transport = NettyTransportFactory.createTransport(uri);
|
||||
transport.setTransportListener(new StompTransportListener());
|
||||
transport.connect();
|
||||
|
||||
Wait.waitFor(() -> transport.isConnected(), 1000);
|
||||
|
||||
if (!transport.isConnected()) {
|
||||
throw new RuntimeException("Could not connect transport");
|
||||
}
|
||||
}
|
||||
|
||||
public void killReaderThread() {
|
||||
readerThread.stop();
|
||||
public AbstractStompClientConnection(URI uri, boolean autoConnect) throws Exception {
|
||||
parseURI(uri);
|
||||
this.factory = StompFrameFactoryFactory.getFactory(version);
|
||||
|
||||
readBuffer = ByteBuffer.allocateDirect(10240);
|
||||
receiveList = new ArrayList<>(10240);
|
||||
|
||||
transport = NettyTransportFactory.createTransport(uri);
|
||||
transport.setTransportListener(new StompTransportListener());
|
||||
|
||||
if (autoConnect) {
|
||||
transport.connect();
|
||||
|
||||
Wait.waitFor(() -> transport.isConnected(), 1000);
|
||||
|
||||
if (!transport.isConnected()) {
|
||||
throw new RuntimeException("Could not connect transport");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void parseURI(URI uri) {
|
||||
scheme = uri.getScheme() == null ? "tcp" : uri.getScheme();
|
||||
host = uri.getHost();
|
||||
port = uri.getPort();
|
||||
this.version = StompClientConnectionFactory.getStompVersionFromURI(uri);
|
||||
}
|
||||
|
||||
private ClientStompFrame sendFrameInternal(ClientStompFrame frame, boolean wicked) throws IOException, InterruptedException {
|
||||
|
@ -85,8 +119,17 @@ public abstract class AbstractStompClientConnection implements StompClientConnec
|
|||
} else {
|
||||
buffer = frame.toByteBuffer();
|
||||
}
|
||||
while (buffer.remaining() > 0) {
|
||||
socketChannel.write(buffer);
|
||||
|
||||
ByteBuf buf = Unpooled.copiedBuffer(buffer);
|
||||
|
||||
try {
|
||||
buf.retain();
|
||||
ChannelFuture future = transport.send(buf);
|
||||
if (future != null) {
|
||||
future.awaitUninterruptibly();
|
||||
}
|
||||
} finally {
|
||||
buf.release();
|
||||
}
|
||||
|
||||
//now response
|
||||
|
@ -179,35 +222,78 @@ public abstract class AbstractStompClientConnection implements StompClientConnec
|
|||
}
|
||||
|
||||
protected void close() throws IOException {
|
||||
socketChannel.close();
|
||||
transport.close();
|
||||
}
|
||||
|
||||
private class ReaderThread extends Thread {
|
||||
private class StompTransportListener implements NettyTransportListener {
|
||||
|
||||
/**
|
||||
* Called when new incoming data has become available.
|
||||
*
|
||||
* @param incoming the next incoming packet of data.
|
||||
*/
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
int n = socketChannel.read(readBuffer);
|
||||
|
||||
while (n >= 0) {
|
||||
if (n > 0) {
|
||||
receiveBytes(n);
|
||||
}
|
||||
n = socketChannel.read(readBuffer);
|
||||
}
|
||||
//peer closed
|
||||
close();
|
||||
|
||||
} catch (IOException e) {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException e1) {
|
||||
//ignore
|
||||
public void onData(ByteBuf incoming) {
|
||||
while (incoming.readableBytes() > 0) {
|
||||
int bytes = incoming.readableBytes();
|
||||
if (incoming.readableBytes() < readBuffer.remaining()) {
|
||||
ByteBuffer byteBuffer = ByteBuffer.allocate(incoming.readableBytes());
|
||||
incoming.readBytes(byteBuffer);
|
||||
byteBuffer.rewind();
|
||||
readBuffer.put(byteBuffer);
|
||||
receiveBytes(bytes);
|
||||
} else {
|
||||
incoming.readBytes(readBuffer);
|
||||
receiveBytes(bytes - incoming.readableBytes());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called if the connection state becomes closed.
|
||||
*/
|
||||
@Override
|
||||
public void onTransportClosed() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when an error occurs during normal Transport operations.
|
||||
*
|
||||
* @param cause the error that triggered this event.
|
||||
*/
|
||||
@Override
|
||||
public void onTransportError(Throwable cause) {
|
||||
throw new RuntimeException(cause);
|
||||
}
|
||||
}
|
||||
|
||||
// private class ReaderThread extends Thread {
|
||||
//
|
||||
// @Override
|
||||
// public void run() {
|
||||
// try {
|
||||
// transport.setTransportListener();
|
||||
// int n = Z..read(readBuffer);
|
||||
//
|
||||
// while (n >= 0) {
|
||||
// if (n > 0) {
|
||||
// receiveBytes(n);
|
||||
// }
|
||||
// n = socketChannel.read(readBuffer);
|
||||
// }
|
||||
// //peer closed
|
||||
// close();
|
||||
//
|
||||
// } catch (IOException e) {
|
||||
// try {
|
||||
// close();
|
||||
// } catch (IOException e1) {
|
||||
// //ignore
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
|
||||
@Override
|
||||
public ClientStompFrame connect() throws Exception {
|
||||
return connect(null, null);
|
||||
|
@ -230,7 +316,7 @@ public abstract class AbstractStompClientConnection implements StompClientConnec
|
|||
|
||||
@Override
|
||||
public boolean isConnected() {
|
||||
return connected && socketChannel.isConnected();
|
||||
return connected && transport.isConnected();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -243,6 +329,16 @@ public abstract class AbstractStompClientConnection implements StompClientConnec
|
|||
return this.frameQueue.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeTransport() throws IOException {
|
||||
transport.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public NettyTransport getTransport() {
|
||||
return transport;
|
||||
}
|
||||
|
||||
protected class Pinger extends Thread {
|
||||
|
||||
long pingInterval;
|
||||
|
|
|
@ -18,6 +18,8 @@ package org.apache.activemq.artemis.tests.integration.stomp.util;
|
|||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
|
||||
/**
|
||||
* pls use factory to create frames.
|
||||
*/
|
||||
|
@ -25,6 +27,8 @@ public interface ClientStompFrame {
|
|||
|
||||
ByteBuffer toByteBuffer();
|
||||
|
||||
ByteBuf toNettyByteBuf();
|
||||
|
||||
boolean needsReply();
|
||||
|
||||
ClientStompFrame setCommand(String command);
|
||||
|
@ -41,6 +45,8 @@ public interface ClientStompFrame {
|
|||
|
||||
ByteBuffer toByteBufferWithExtra(String str);
|
||||
|
||||
ByteBuf toNettyByteBufWithExtras(String str);
|
||||
|
||||
boolean isPing();
|
||||
|
||||
ClientStompFrame setForceOneway();
|
||||
|
|
|
@ -18,6 +18,8 @@ package org.apache.activemq.artemis.tests.integration.stomp.util;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.activemq.transport.netty.NettyTransport;
|
||||
|
||||
public interface StompClientConnection {
|
||||
|
||||
ClientStompFrame sendFrame(ClientStompFrame frame) throws IOException, InterruptedException;
|
||||
|
@ -53,5 +55,8 @@ public interface StompClientConnection {
|
|||
|
||||
int getServerPingNumber();
|
||||
|
||||
void closeTransport() throws IOException;
|
||||
|
||||
NettyTransport getTransport();
|
||||
}
|
||||
|
||||
|
|
|
@ -17,9 +17,12 @@
|
|||
package org.apache.activemq.artemis.tests.integration.stomp.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
|
||||
public class StompClientConnectionFactory {
|
||||
|
||||
public static final String LATEST_VERSION = "1.2";
|
||||
|
||||
//create a raw connection to the host.
|
||||
public static StompClientConnection createClientConnection(String version,
|
||||
String host,
|
||||
|
@ -36,6 +39,48 @@ public class StompClientConnectionFactory {
|
|||
return null;
|
||||
}
|
||||
|
||||
public static StompClientConnection createClientConnection(URI uri) throws Exception {
|
||||
String version = getStompVersionFromURI(uri);
|
||||
if ("1.0".equals(version)) {
|
||||
return new StompClientConnectionV10(uri);
|
||||
}
|
||||
if ("1.1".equals(version)) {
|
||||
return new StompClientConnectionV11(uri);
|
||||
}
|
||||
if ("1.2".equals(version)) {
|
||||
return new StompClientConnectionV12(uri);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public static StompClientConnection createClientConnection(URI uri, boolean autoConnect) throws Exception {
|
||||
String version = getStompVersionFromURI(uri);
|
||||
if ("1.0".equals(version)) {
|
||||
return new StompClientConnectionV10(uri, autoConnect);
|
||||
}
|
||||
if ("1.1".equals(version)) {
|
||||
return new StompClientConnectionV11(uri, autoConnect);
|
||||
}
|
||||
if ("1.2".equals(version)) {
|
||||
return new StompClientConnectionV12(uri, autoConnect);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public static String getStompVersionFromURI(URI uri) {
|
||||
String scheme = uri.getScheme();
|
||||
if (scheme.contains("10")) {
|
||||
return "1.0";
|
||||
}
|
||||
if (scheme.contains("11")) {
|
||||
return "1.1";
|
||||
}
|
||||
if (scheme.contains("12")) {
|
||||
return "1.2";
|
||||
}
|
||||
return LATEST_VERSION;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
StompClientConnection connection = StompClientConnectionFactory.createClientConnection("1.0", "localhost", 61613);
|
||||
|
||||
|
|
|
@ -17,12 +17,14 @@
|
|||
package org.apache.activemq.artemis.tests.integration.stomp.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.Stomp;
|
||||
import org.apache.activemq.artemis.tests.integration.IntegrationTestLogger;
|
||||
|
||||
public class StompClientConnectionV10 extends AbstractStompClientConnection {
|
||||
|
||||
|
||||
public StompClientConnectionV10(String host, int port) throws IOException {
|
||||
super("1.0", host, port);
|
||||
}
|
||||
|
@ -31,6 +33,14 @@ public class StompClientConnectionV10 extends AbstractStompClientConnection {
|
|||
super(version, host, port);
|
||||
}
|
||||
|
||||
public StompClientConnectionV10(URI uri) throws Exception {
|
||||
super(uri);
|
||||
}
|
||||
|
||||
public StompClientConnectionV10(URI uri, boolean autoConnect) throws Exception {
|
||||
super(uri, autoConnect);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClientStompFrame connect(String username, String passcode) throws IOException, InterruptedException {
|
||||
return connect(username, passcode, null);
|
||||
|
@ -38,6 +48,7 @@ public class StompClientConnectionV10 extends AbstractStompClientConnection {
|
|||
|
||||
@Override
|
||||
public ClientStompFrame connect(String username, String passcode, String clientID) throws IOException, InterruptedException {
|
||||
|
||||
ClientStompFrame frame = factory.newFrame(Stomp.Commands.CONNECT);
|
||||
frame.addHeader(Stomp.Headers.Connect.LOGIN, username);
|
||||
frame.addHeader(Stomp.Headers.Connect.PASSCODE, passcode);
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.activemq.artemis.tests.integration.stomp.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.Stomp;
|
||||
|
@ -31,6 +32,14 @@ public class StompClientConnectionV11 extends StompClientConnectionV10 {
|
|||
super(version, host, port);
|
||||
}
|
||||
|
||||
public StompClientConnectionV11(URI uri) throws Exception {
|
||||
super(uri);
|
||||
}
|
||||
|
||||
public StompClientConnectionV11(URI uri, boolean autoConnect) throws Exception {
|
||||
super(uri, autoConnect);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClientStompFrame connect(String username, String passcode, String clientID) throws IOException, InterruptedException {
|
||||
ClientStompFrame frame = factory.newFrame(Stomp.Commands.CONNECT);
|
||||
|
@ -96,12 +105,16 @@ public class StompClientConnectionV11 extends StompClientConnectionV10 {
|
|||
|
||||
frame.addHeader(Stomp.Headers.RECEIPT_REQUESTED, uuid);
|
||||
|
||||
ClientStompFrame result = this.sendFrame(frame);
|
||||
|
||||
if (result == null || (!Stomp.Responses.RECEIPT.equals(result.getCommand())) || (!uuid.equals(result.getHeader(Stomp.Headers.Response.RECEIPT_ID)))) {
|
||||
throw new IOException("Disconnect failed! " + result);
|
||||
try {
|
||||
if (!transport.isConnected()) {
|
||||
ClientStompFrame result = this.sendFrame(frame);
|
||||
if (result == null || (!Stomp.Responses.RECEIPT.equals(result.getCommand())) || (!uuid.equals(result.getHeader(Stomp.Headers.Response.RECEIPT_ID)))) {
|
||||
throw new IOException("Disconnect failed! " + result);
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// Transport may have been closed
|
||||
}
|
||||
|
||||
close();
|
||||
|
||||
connected = false;
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.activemq.artemis.tests.integration.stomp.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
|
||||
public class StompClientConnectionV12 extends StompClientConnectionV11 {
|
||||
|
||||
|
@ -24,6 +25,14 @@ public class StompClientConnectionV12 extends StompClientConnectionV11 {
|
|||
super("1.2", host, port);
|
||||
}
|
||||
|
||||
public StompClientConnectionV12(URI uri) throws Exception {
|
||||
super(uri);
|
||||
}
|
||||
|
||||
public StompClientConnectionV12(URI uri, boolean autoConnect) throws Exception {
|
||||
super(uri, autoConnect);
|
||||
}
|
||||
|
||||
public ClientStompFrame createAnyFrame(String command) {
|
||||
return factory.newAnyFrame(command);
|
||||
}
|
||||
|
|
|
@ -16,7 +16,10 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.tests.integration.stomp.v11;
|
||||
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.Stomp;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.StompTestBase;
|
||||
|
@ -26,15 +29,23 @@ import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConne
|
|||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
/*
|
||||
* Some Stomp tests against server with persistence enabled are put here.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public class ExtraStompTest extends StompTestBase {
|
||||
|
||||
private StompClientConnection connV10;
|
||||
private StompClientConnection connV11;
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> data() {
|
||||
return Arrays.asList(new Object[][]{{"ws+v11.stomp"}, {"tcp+v11.stomp"}});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPersistenceEnabled() {
|
||||
return true;
|
||||
|
@ -44,9 +55,11 @@ public class ExtraStompTest extends StompTestBase {
|
|||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
connV10 = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
URI v10Uri = new URI(uri.toString().replace("v11", "v10"));
|
||||
connV10 = StompClientConnectionFactory.createClientConnection(v10Uri);
|
||||
connV10.connect(defUser, defPass);
|
||||
connV11 = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
|
||||
connV11 = StompClientConnectionFactory.createClientConnection(uri);
|
||||
connV11.connect(defUser, defPass);
|
||||
}
|
||||
|
||||
|
@ -181,17 +194,19 @@ public class ExtraStompTest extends StompTestBase {
|
|||
|
||||
conn.sendFrame(frame);
|
||||
|
||||
subscribe(conn, "a-sub", Stomp.Headers.Subscribe.AckModeValues.CLIENT);
|
||||
frame = subscribe(conn, "a-sub", Stomp.Headers.Subscribe.AckModeValues.CLIENT);
|
||||
|
||||
// receive but don't ack
|
||||
frame = conn.receiveFrame(10000);
|
||||
System.out.println(frame);
|
||||
|
||||
frame = conn.receiveFrame(10000);
|
||||
System.out.println(frame);
|
||||
|
||||
unsubscribe(conn, "a-sub");
|
||||
|
||||
subscribe(conn, "a-sub");
|
||||
frame = subscribe(conn, "a-sub");
|
||||
|
||||
frame = conn.receiveFrame(10000);
|
||||
frame = conn.receiveFrame(10000);
|
||||
|
||||
//second receive will get problem if trailing bytes
|
||||
|
|
|
@ -23,20 +23,22 @@ import javax.jms.MessageListener;
|
|||
import javax.jms.MessageProducer;
|
||||
import javax.jms.TextMessage;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.Stomp;
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.StompConnection;
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.v11.StompFrameHandlerV11;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.tests.integration.IntegrationTestLogger;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.StompTestBase;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.AbstractStompClientConnection;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.ClientStompFrame;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConnection;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.util.StompClientConnectionFactory;
|
||||
|
@ -46,10 +48,13 @@ import org.junit.After;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
/*
|
||||
*
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public class StompV11Test extends StompTestBase {
|
||||
|
||||
private static final transient IntegrationTestLogger log = IntegrationTestLogger.LOGGER;
|
||||
|
@ -57,11 +62,19 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
private StompClientConnection conn;
|
||||
|
||||
private URI v10Uri;
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> data() {
|
||||
return Arrays.asList(new Object[][]{{"ws+v11.stomp"}, {"tcp+v11.stomp"}});
|
||||
}
|
||||
|
||||
@Override
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
v10Uri = new URI(uri.toString().replace("v11", "v10"));
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -75,13 +88,14 @@ public class StompV11Test extends StompTestBase {
|
|||
}
|
||||
} finally {
|
||||
super.tearDown();
|
||||
conn.closeTransport();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConnection() throws Exception {
|
||||
server.getActiveMQServer().getConfiguration().setSecurityEnabled(true);
|
||||
StompClientConnection connection = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection connection = StompClientConnectionFactory.createClientConnection(v10Uri);
|
||||
|
||||
connection.connect(defUser, defPass);
|
||||
|
||||
|
@ -91,7 +105,7 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
connection.disconnect();
|
||||
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
|
||||
connection.connect(defUser, defPass);
|
||||
|
||||
|
@ -101,14 +115,14 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
connection.disconnect();
|
||||
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
|
||||
connection.connect();
|
||||
|
||||
assertFalse(connection.isConnected());
|
||||
|
||||
//new way of connection
|
||||
StompClientConnectionV11 conn = (StompClientConnectionV11) StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnectionV11 conn = (StompClientConnectionV11) StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect1(defUser, defPass);
|
||||
|
||||
assertTrue(conn.isConnected());
|
||||
|
@ -116,7 +130,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
//invalid user
|
||||
conn = (StompClientConnectionV11) StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = (StompClientConnectionV11) StompClientConnectionFactory.createClientConnection(uri);
|
||||
ClientStompFrame frame = conn.connect("invaliduser", defPass);
|
||||
assertFalse(conn.isConnected());
|
||||
assertTrue(Stomp.Responses.ERROR.equals(frame.getCommand()));
|
||||
|
@ -141,7 +155,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
// case 2 accept-version=1.0, result: 1.0
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.ACCEPT_VERSION, "1.0")
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
|
@ -158,7 +172,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
// case 3 accept-version=1.1, result: 1.1
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.ACCEPT_VERSION, "1.1")
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
|
@ -175,7 +189,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
// case 4 accept-version=1.0,1.1,1.2, result 1.1
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.ACCEPT_VERSION, "1.0,1.1,1.3")
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
|
@ -192,7 +206,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
// case 5 accept-version=1.2, result error
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.ACCEPT_VERSION, "1.3")
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
|
@ -220,7 +234,7 @@ public class StompV11Test extends StompTestBase {
|
|||
response = send(conn, getQueuePrefix() + getQueueName(), "text/plain", "Hello World 2!", true);
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
|
||||
subscribe(newConn, "a-sub");
|
||||
|
@ -254,7 +268,7 @@ public class StompV11Test extends StompTestBase {
|
|||
send(conn, getQueuePrefix() + getQueueName(), "application/xml", "Hello World 1!");
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
|
||||
subscribe(newConn, "a-sub");
|
||||
|
@ -289,7 +303,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.sendFrame(frame);
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
|
||||
subscribe(newConn, "a-sub");
|
||||
|
@ -330,7 +344,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.sendFrame(frame);
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
|
||||
subscribe(newConn, "a-sub");
|
||||
|
@ -365,6 +379,7 @@ public class StompV11Test extends StompTestBase {
|
|||
frame.addHeader("destination", getQueuePrefix() + getQueueName());
|
||||
frame.addHeader("content-type", "text/plain");
|
||||
frame.addHeader("content-length", cLen);
|
||||
//frame.addHeader(Stomp.Headers.Connect.HEART_BEAT, "0,0");
|
||||
String hKey = "undefined-escape";
|
||||
String hVal = "is\\ttab";
|
||||
frame.addHeader(hKey, hVal);
|
||||
|
@ -403,7 +418,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
//default heart beat for (0,0) which is default connection TTL (60000) / default heartBeatToTtlModifier (2.0) = 30000
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -424,7 +439,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
//heart-beat (1,0), should receive a min client ping accepted by server
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -450,7 +465,7 @@ public class StompV11Test extends StompTestBase {
|
|||
}
|
||||
|
||||
//heart-beat (1,0), start a ping, then send a message, should be ok.
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -499,7 +514,7 @@ public class StompV11Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
//heart-beat (500,1000)
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -554,7 +569,7 @@ public class StompV11Test extends StompTestBase {
|
|||
}
|
||||
|
||||
// subscribe
|
||||
newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
|
||||
subscribe(newConn, "a-sub");
|
||||
|
@ -590,7 +605,7 @@ public class StompV11Test extends StompTestBase {
|
|||
}
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
try {
|
||||
ClientStompFrame frame = newConn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
|
@ -647,7 +662,7 @@ public class StompV11Test extends StompTestBase {
|
|||
}
|
||||
|
||||
// subscribe
|
||||
newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = newConn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -689,8 +704,10 @@ public class StompV11Test extends StompTestBase {
|
|||
ClientStompFrame reply;
|
||||
int port = 61614;
|
||||
|
||||
uri = createStompClientUri(scheme, hostname, port);
|
||||
|
||||
server.getActiveMQServer().getRemotingService().createAcceptor("test", "tcp://127.0.0.1:" + port + "?connectionTtl=1000&connectionTtlMin=5000&connectionTtlMax=10000").start();
|
||||
StompClientConnection connection = StompClientConnectionFactory.createClientConnection("1.1", "localhost", port);
|
||||
StompClientConnection connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
|
||||
//no heart beat at all if heat-beat absent
|
||||
frame = connection.createFrame(Stomp.Commands.CONNECT)
|
||||
|
@ -709,14 +726,15 @@ public class StompV11Test extends StompTestBase {
|
|||
assertEquals(0, connection.getFrameQueueSize());
|
||||
|
||||
try {
|
||||
connection.disconnect();
|
||||
fail("Channel should be closed here already due to TTL");
|
||||
assertFalse(connection.isConnected());
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
} finally {
|
||||
connection.closeTransport();
|
||||
}
|
||||
|
||||
//no heart beat for (0,0)
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.1", "localhost", port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = connection.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -739,14 +757,15 @@ public class StompV11Test extends StompTestBase {
|
|||
assertEquals(0, connection.getFrameQueueSize());
|
||||
|
||||
try {
|
||||
connection.disconnect();
|
||||
fail("Channel should be closed here already due to TTL");
|
||||
assertFalse(connection.isConnected());
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
} finally {
|
||||
connection.closeTransport();
|
||||
}
|
||||
|
||||
//heart-beat (1,0), should receive a min client ping accepted by server
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.1", "localhost", port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = connection.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -765,14 +784,15 @@ public class StompV11Test extends StompTestBase {
|
|||
//now server side should be disconnected because we didn't send ping for 2 sec
|
||||
//send will fail
|
||||
try {
|
||||
send(connection, getQueuePrefix() + getQueueName(), "text/plain", "Hello World");
|
||||
fail("connection should have been destroyed by now");
|
||||
} catch (IOException e) {
|
||||
//ignore
|
||||
assertFalse(connection.isConnected());
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
} finally {
|
||||
connection.closeTransport();
|
||||
}
|
||||
|
||||
//heart-beat (1,0), start a ping, then send a message, should be ok.
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.1", "localhost", port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = connection.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -801,7 +821,7 @@ public class StompV11Test extends StompTestBase {
|
|||
connection.disconnect();
|
||||
|
||||
//heart-beat (20000,0), should receive a max client ping accepted by server
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.1", "localhost", port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = connection.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -820,10 +840,11 @@ public class StompV11Test extends StompTestBase {
|
|||
//now server side should be disconnected because we didn't send ping for 2 sec
|
||||
//send will fail
|
||||
try {
|
||||
send(connection, getQueuePrefix() + getQueueName(), "text/plain", "Hello World");
|
||||
fail("connection should have been destroyed by now");
|
||||
} catch (IOException e) {
|
||||
//ignore
|
||||
assertFalse(connection.isConnected());
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
} finally {
|
||||
connection.closeTransport();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -836,7 +857,7 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
server.getActiveMQServer().getRemotingService().createAcceptor("test", "tcp://127.0.0.1:" + port + "?heartBeatToConnectionTtlModifier=1").start();
|
||||
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.1", "localhost", port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = connection.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -853,16 +874,15 @@ public class StompV11Test extends StompTestBase {
|
|||
Thread.sleep(6000);
|
||||
|
||||
try {
|
||||
connection.disconnect();
|
||||
fail("Connection should be closed here already due to TTL");
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
assertFalse(connection.isConnected());
|
||||
} finally {
|
||||
connection.closeTransport();
|
||||
}
|
||||
|
||||
server.getActiveMQServer().getRemotingService().destroyAcceptor("test");
|
||||
server.getActiveMQServer().getRemotingService().createAcceptor("test", "tcp://127.0.0.1:" + port + "?heartBeatToConnectionTtlModifier=1.5").start();
|
||||
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.1", "localhost", port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = connection.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -1151,6 +1171,7 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
subscribe(conn, "sub1", Stomp.Headers.Subscribe.AckModeValues.CLIENT);
|
||||
|
||||
Thread.sleep(1000);
|
||||
int num = 50;
|
||||
//send a bunch of messages
|
||||
for (int i = 0; i < num; i++) {
|
||||
|
@ -1175,7 +1196,7 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
//no messages can be received.
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
Message message = consumer.receive(1000);
|
||||
Message message = consumer.receive(10000);
|
||||
Assert.assertNotNull(message);
|
||||
message = consumer.receive(1000);
|
||||
Assert.assertNull(message);
|
||||
|
@ -1260,21 +1281,21 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
this.subscribeTopic(conn, "sub1", Stomp.Headers.Subscribe.AckModeValues.AUTO, null);
|
||||
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass, "myclientid2");
|
||||
|
||||
this.subscribeTopic(newConn, "sub2", Stomp.Headers.Subscribe.AckModeValues.AUTO, null);
|
||||
|
||||
send(conn, getTopicPrefix() + getTopicName(), null, "Hello World");
|
||||
send(newConn, getTopicPrefix() + getTopicName(), null, "Hello World");
|
||||
|
||||
// receive message from socket
|
||||
ClientStompFrame frame = conn.receiveFrame(1000);
|
||||
ClientStompFrame frame = conn.receiveFrame(5000);
|
||||
|
||||
IntegrationTestLogger.LOGGER.info("received frame : " + frame);
|
||||
assertEquals("Hello World", frame.getBody());
|
||||
assertEquals("sub1", frame.getHeader(Stomp.Headers.Message.SUBSCRIPTION));
|
||||
|
||||
frame = newConn.receiveFrame(1000);
|
||||
frame = newConn.receiveFrame(5000);
|
||||
|
||||
IntegrationTestLogger.LOGGER.info("received 2 frame : " + frame);
|
||||
assertEquals("Hello World", frame.getBody());
|
||||
|
@ -1294,7 +1315,7 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
send(conn, getQueuePrefix() + getQueueName(), null, "Hello World");
|
||||
|
||||
StompClientConnection connV11_2 = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection connV11_2 = StompClientConnectionFactory.createClientConnection(uri);
|
||||
connV11_2.connect(defUser, defPass);
|
||||
|
||||
this.subscribe(connV11_2, "sub1", Stomp.Headers.Subscribe.AckModeValues.AUTO);
|
||||
|
@ -1434,9 +1455,9 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
this.subscribe(conn, "sub1", Stomp.Headers.Subscribe.AckModeValues.CLIENT, getName());
|
||||
|
||||
this.subscribe(conn, "sub1", Stomp.Headers.Subscribe.AckModeValues.CLIENT, getName());
|
||||
|
||||
this.subscribe(conn, "sub1", Stomp.Headers.Subscribe.AckModeValues.CLIENT, getName(), false);
|
||||
ClientStompFrame frame = conn.receiveFrame();
|
||||
|
||||
Assert.assertTrue(frame.getCommand().equals(Stomp.Responses.ERROR));
|
||||
|
||||
conn.disconnect();
|
||||
|
@ -1463,7 +1484,7 @@ public class StompV11Test extends StompTestBase {
|
|||
sendJmsMessage(getName(), topic);
|
||||
|
||||
conn.destroy();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass, CLIENT_ID);
|
||||
|
||||
this.subscribeTopic(conn, "sub1", Stomp.Headers.Subscribe.AckModeValues.AUTO, getName());
|
||||
|
@ -1488,7 +1509,7 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
conn.disconnect();
|
||||
conn.destroy();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass, CLIENT_ID);
|
||||
|
||||
this.unsubscribe(conn, getName(), null, false, true);
|
||||
|
@ -1689,6 +1710,7 @@ public class StompV11Test extends StompTestBase {
|
|||
@Test
|
||||
public void testSendMessageWithLeadingNewLine() throws Exception {
|
||||
MessageConsumer consumer = session.createConsumer(queue);
|
||||
Thread.sleep(1000);
|
||||
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
|
@ -2151,7 +2173,7 @@ public class StompV11Test extends StompTestBase {
|
|||
int size = conn.getServerPingNumber();
|
||||
|
||||
conn.stopPinger();
|
||||
((AbstractStompClientConnection)conn).killReaderThread();
|
||||
//((AbstractStompClientConnection)conn).killReaderThread();
|
||||
Wait.waitFor(() -> {
|
||||
return server.getActiveMQServer().getRemotingService().getConnections().size() == 0;
|
||||
});
|
||||
|
@ -2175,10 +2197,10 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
if (sendDisconnect) {
|
||||
conn.disconnect();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
} else {
|
||||
conn.destroy();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
}
|
||||
|
||||
// message should be received since message was not acknowledged
|
||||
|
@ -2193,7 +2215,7 @@ public class StompV11Test extends StompTestBase {
|
|||
|
||||
// now let's make sure we don't see the message again
|
||||
conn.destroy();
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
this.subscribe(conn, "sub1", null, null, true);
|
||||
|
|
|
@ -24,15 +24,18 @@ import javax.jms.MessageListener;
|
|||
import javax.jms.MessageProducer;
|
||||
import javax.jms.TextMessage;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.protocol.stomp.Stomp;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
|
||||
import org.apache.activemq.artemis.tests.integration.IntegrationTestLogger;
|
||||
import org.apache.activemq.artemis.tests.integration.stomp.StompTestBase;
|
||||
|
@ -45,6 +48,7 @@ import org.junit.After;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
/**
|
||||
* Testing Stomp version 1.2 functionalities
|
||||
|
@ -56,11 +60,22 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
private StompClientConnectionV12 conn;
|
||||
|
||||
private URI v10Uri;
|
||||
|
||||
private URI v11Uri;
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> data() {
|
||||
return Arrays.asList(new Object[][]{{"ws+v12.stomp"}, {"tcp+v12.stomp"}});
|
||||
}
|
||||
|
||||
@Override
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
v10Uri = new URI(uri.toString().replace("v12", "v10"));
|
||||
v11Uri = new URI(uri.toString().replace("v12", "v11"));
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection(uri);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -74,13 +89,14 @@ public class StompV12Test extends StompTestBase {
|
|||
}
|
||||
} finally {
|
||||
super.tearDown();
|
||||
conn.closeTransport();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConnection() throws Exception {
|
||||
server.getActiveMQServer().getConfiguration().setSecurityEnabled(true);
|
||||
StompClientConnection connection = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection connection = StompClientConnectionFactory.createClientConnection(v10Uri);
|
||||
|
||||
connection.connect(defUser, defPass);
|
||||
|
||||
|
@ -90,7 +106,7 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
connection.disconnect();
|
||||
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
|
||||
connection.connect(defUser, defPass);
|
||||
|
||||
|
@ -100,14 +116,14 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
connection.disconnect();
|
||||
|
||||
connection = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
connection = StompClientConnectionFactory.createClientConnection(uri);
|
||||
|
||||
connection.connect();
|
||||
|
||||
Assert.assertFalse(connection.isConnected());
|
||||
|
||||
//new way of connection
|
||||
StompClientConnectionV11 conn = (StompClientConnectionV11) StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnectionV11 conn = (StompClientConnectionV11) StompClientConnectionFactory.createClientConnection(v11Uri);
|
||||
conn.connect1(defUser, defPass);
|
||||
|
||||
Assert.assertTrue(conn.isConnected());
|
||||
|
@ -117,7 +133,7 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
@Test
|
||||
public void testConnectionAsInSpec() throws Exception {
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(v10Uri);
|
||||
|
||||
ClientStompFrame frame = conn.createFrame(Stomp.Commands.CONNECT);
|
||||
frame.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser);
|
||||
|
@ -133,7 +149,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
//need 1.2 client
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
|
||||
frame = conn.createFrame(Stomp.Commands.STOMP);
|
||||
frame.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser);
|
||||
|
@ -151,7 +167,7 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
@Test
|
||||
public void testNegotiation() throws Exception {
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.0", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(v10Uri);
|
||||
// case 1 accept-version absent. It is a 1.0 connect
|
||||
ClientStompFrame frame = conn.createFrame(Stomp.Commands.CONNECT);
|
||||
frame.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1");
|
||||
|
@ -168,7 +184,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
// case 2 accept-version=1.0, result: 1.0
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(v11Uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT);
|
||||
frame.addHeader(Stomp.Headers.ACCEPT_VERSION, "1.0");
|
||||
frame.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1");
|
||||
|
@ -185,7 +201,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
// case 3 accept-version=1.1, result: 1.1
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(v11Uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT);
|
||||
frame.addHeader(Stomp.Headers.ACCEPT_VERSION, "1.1");
|
||||
frame.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1");
|
||||
|
@ -202,7 +218,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
// case 4 accept-version=1.0,1.1,1.3, result 1.2
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(v11Uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT);
|
||||
frame.addHeader(Stomp.Headers.ACCEPT_VERSION, "1.0,1.1,1.3");
|
||||
frame.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1");
|
||||
|
@ -219,7 +235,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
// case 5 accept-version=1.3, result error
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(v11Uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT);
|
||||
frame.addHeader(Stomp.Headers.ACCEPT_VERSION, "1.3");
|
||||
frame.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1");
|
||||
|
@ -230,6 +246,8 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
Assert.assertEquals(Stomp.Responses.ERROR, reply.getCommand());
|
||||
|
||||
conn.disconnect();
|
||||
|
||||
System.out.println("Got error frame " + reply);
|
||||
|
||||
}
|
||||
|
@ -245,7 +263,7 @@ public class StompV12Test extends StompTestBase {
|
|||
send(conn, getQueuePrefix() + getQueueName(), "text/plain", "Hello World 2!", true);
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
subscribe(newConn, "a-sub");
|
||||
|
||||
|
@ -281,7 +299,7 @@ public class StompV12Test extends StompTestBase {
|
|||
send(conn, getQueuePrefix() + getQueueName(), "application/xml", "Hello World 1!");
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(v11Uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
subscribe(newConn, "a-sub");
|
||||
|
||||
|
@ -315,7 +333,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.sendFrame(frame);
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
subscribe(newConn, "a-sub");
|
||||
|
||||
|
@ -376,7 +394,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.sendFrame(frame);
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
subscribe(newConn, "a-sub", null, null, true);
|
||||
|
||||
|
@ -434,7 +452,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.sendFrame(frame);
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
subscribe(newConn, "a-sub");
|
||||
|
||||
|
@ -481,7 +499,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.sendFrame(frame);
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
|
||||
subscribe(newConn, "a-sub");
|
||||
|
@ -540,7 +558,7 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
@Test
|
||||
public void testHeartBeat() throws Exception {
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
StompClientConnection conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
//no heart beat at all if heat-beat absent
|
||||
ClientStompFrame frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
|
@ -558,7 +576,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
//no heart beat for (0,0)
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -579,7 +597,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
//heart-beat (1,0), should receive a min client ping accepted by server
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -605,7 +623,7 @@ public class StompV12Test extends StompTestBase {
|
|||
}
|
||||
|
||||
//heart-beat (1,0), start a ping, then send a message, should be ok.
|
||||
conn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -650,7 +668,7 @@ public class StompV12Test extends StompTestBase {
|
|||
conn.disconnect();
|
||||
|
||||
//heart-beat (500,1000)
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = conn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -703,7 +721,7 @@ public class StompV12Test extends StompTestBase {
|
|||
}
|
||||
|
||||
// subscribe
|
||||
newConn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
newConn.connect(defUser, defPass);
|
||||
subscribe(newConn, "a-sub");
|
||||
|
||||
|
@ -738,7 +756,7 @@ public class StompV12Test extends StompTestBase {
|
|||
}
|
||||
|
||||
//subscribe
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(v11Uri);
|
||||
try {
|
||||
ClientStompFrame frame = newConn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
|
@ -795,7 +813,7 @@ public class StompV12Test extends StompTestBase {
|
|||
}
|
||||
|
||||
// subscribe
|
||||
newConn = StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
newConn = StompClientConnectionFactory.createClientConnection(uri);
|
||||
frame = newConn.createFrame(Stomp.Commands.CONNECT)
|
||||
.addHeader(Stomp.Headers.Connect.HOST, "127.0.0.1")
|
||||
.addHeader(Stomp.Headers.Connect.LOGIN, this.defUser)
|
||||
|
@ -1250,7 +1268,7 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
this.subscribeTopic(conn, "sub1", Stomp.Headers.Subscribe.AckModeValues.AUTO, null);
|
||||
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection newConn = StompClientConnectionFactory.createClientConnection(v11Uri);
|
||||
newConn.connect(defUser, defPass, "myclientid2");
|
||||
|
||||
this.subscribeTopic(newConn, "sub2", Stomp.Headers.Subscribe.AckModeValues.AUTO, null);
|
||||
|
@ -1284,7 +1302,7 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
send(conn, getQueuePrefix() + getQueueName(), null, "Hello World");
|
||||
|
||||
StompClientConnection connV12_2 = StompClientConnectionFactory.createClientConnection("1.1", hostname, port);
|
||||
StompClientConnection connV12_2 = StompClientConnectionFactory.createClientConnection(v11Uri);
|
||||
connV12_2.connect(defUser, defPass);
|
||||
|
||||
this.subscribe(connV12_2, "sub1", Stomp.Headers.Subscribe.AckModeValues.AUTO);
|
||||
|
@ -1423,9 +1441,8 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
this.subscribe(conn, "sub1", "client", getName());
|
||||
|
||||
this.subscribe(conn, "sub1", "client", getName());
|
||||
ClientStompFrame frame = this.subscribe(conn, "sub1", "client", getName());
|
||||
|
||||
ClientStompFrame frame = conn.receiveFrame();
|
||||
Assert.assertTrue(frame.getCommand().equals(Stomp.Responses.ERROR));
|
||||
|
||||
waitDisconnect(conn);
|
||||
|
@ -1451,7 +1468,7 @@ public class StompV12Test extends StompTestBase {
|
|||
sendJmsMessage(getName(), topic);
|
||||
|
||||
conn.destroy();
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass, CLIENT_ID);
|
||||
|
||||
this.subscribeTopic(conn, "sub1", Stomp.Headers.Subscribe.AckModeValues.AUTO, getName());
|
||||
|
@ -1476,7 +1493,7 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
conn.disconnect();
|
||||
conn.destroy();
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass, CLIENT_ID);
|
||||
|
||||
this.unsubscribe(conn, getName(), null, false, true);
|
||||
|
@ -2131,7 +2148,7 @@ public class StompV12Test extends StompTestBase {
|
|||
sendJmsMessage("second message");
|
||||
|
||||
//reconnect
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
frame = conn.receiveFrame(1000);
|
||||
|
@ -2172,10 +2189,10 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
if (sendDisconnect) {
|
||||
conn.disconnect();
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection(uri);
|
||||
} else {
|
||||
conn.destroy();
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection(uri);
|
||||
}
|
||||
|
||||
// message should be received since message was not acknowledged
|
||||
|
@ -2190,7 +2207,7 @@ public class StompV12Test extends StompTestBase {
|
|||
|
||||
// now let's make sure we don't see the message again
|
||||
conn.destroy();
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection("1.2", hostname, port);
|
||||
conn = (StompClientConnectionV12) StompClientConnectionFactory.createClientConnection(uri);
|
||||
conn.connect(defUser, defPass);
|
||||
|
||||
this.subscribe(conn, "sub1", null, null, true);
|
||||
|
|
|
@ -25,9 +25,9 @@ import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
|||
import org.apache.activemq.artemis.core.server.ActiveMQServers;
|
||||
import org.apache.activemq.artemis.junit.Wait;
|
||||
import org.apache.activemq.artemis.tests.util.ActiveMQTestBase;
|
||||
import org.apache.activemq.transport.amqp.client.transport.NettyTransport;
|
||||
import org.apache.activemq.transport.amqp.client.transport.NettyTransportFactory;
|
||||
import org.apache.activemq.transport.amqp.client.transport.NettyTransportListener;
|
||||
import org.apache.activemq.transport.netty.NettyTransport;
|
||||
import org.apache.activemq.transport.netty.NettyTransportFactory;
|
||||
import org.apache.activemq.transport.netty.NettyTransportListener;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.net.URI;
|
||||
|
|
Loading…
Reference in New Issue