ARTEMIS-1780 Handle conversion of large ObjectMessage types

Make sure the correct buffer is used when decoding the stored Core
message that originated from the conversion of an AMQP message sent and
annotated as a JMS ObjectMessage which trips the large message boundary.
This commit is contained in:
Timothy Bish 2018-04-02 17:57:54 -04:00 committed by Clebert Suconic
parent 32090121d2
commit fc32bc0b07
3 changed files with 168 additions and 50 deletions

View File

@ -17,14 +17,31 @@
package org.apache.activemq.artemis.protocol.amqp.converter;
import javax.jms.Destination;
import javax.jms.JMSException;
import javax.jms.MessageEOFException;
import javax.jms.Queue;
import javax.jms.TemporaryQueue;
import javax.jms.TemporaryTopic;
import javax.jms.TextMessage;
import javax.jms.Topic;
import static org.apache.activemq.artemis.api.core.FilterConstants.NATIVE_MESSAGE_ID;
import static org.apache.activemq.artemis.api.core.Message.HDR_SCHEDULED_DELIVERY_TIME;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.EMPTY_BINARY;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_CONTENT_ENCODING;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_CONTENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_DELIVERY_ANNOTATION_PREFIX;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_FIRST_ACQUIRER;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_FOOTER_PREFIX;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER_DURABLE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER_PRIORITY;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_MESSAGE_ANNOTATION_PREFIX;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_NATIVE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_PREFIX;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_PROPERTIES;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_REPLYTO_GROUP_ID;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_DEST_TYPE_MSG_ANNOTATION;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_REPLY_TO_TYPE_MSG_ANNOTATION;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.QUEUE_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TEMP_QUEUE_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TEMP_TOPIC_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TOPIC_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.toAddress;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Date;
@ -34,8 +51,15 @@ import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.PooledByteBufAllocator;
import javax.jms.Destination;
import javax.jms.JMSException;
import javax.jms.MessageEOFException;
import javax.jms.Queue;
import javax.jms.TemporaryQueue;
import javax.jms.TemporaryTopic;
import javax.jms.TextMessage;
import javax.jms.Topic;
import org.apache.activemq.artemis.api.core.ICoreMessage;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
@ -67,30 +91,8 @@ import org.apache.qpid.proton.codec.EncoderImpl;
import org.apache.qpid.proton.codec.WritableBuffer;
import org.jboss.logging.Logger;
import static org.apache.activemq.artemis.api.core.FilterConstants.NATIVE_MESSAGE_ID;
import static org.apache.activemq.artemis.api.core.Message.HDR_SCHEDULED_DELIVERY_TIME;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.EMPTY_BINARY;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_CONTENT_ENCODING;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_CONTENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_DELIVERY_ANNOTATION_PREFIX;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_FIRST_ACQUIRER;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_FOOTER_PREFIX;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER_DURABLE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER_PRIORITY;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_MESSAGE_ANNOTATION_PREFIX;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_NATIVE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_PREFIX;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_PROPERTIES;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_REPLYTO_GROUP_ID;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_DEST_TYPE_MSG_ANNOTATION;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_REPLY_TO_TYPE_MSG_ANNOTATION;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.QUEUE_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TEMP_QUEUE_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TEMP_TOPIC_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TOPIC_TYPE;
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.toAddress;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.PooledByteBufAllocator;
public class CoreAmqpConverter {
@ -405,12 +407,7 @@ public class CoreAmqpConverter {
}
private static Binary getBinaryFromMessageBody(ServerJMSObjectMessage message) throws JMSException {
message.getInnerMessage().getBodyBuffer().resetReaderIndex();
int size = message.getInnerMessage().getBodyBuffer().readInt();
byte[] bytes = new byte[size];
message.getInnerMessage().getBodyBuffer().readBytes(bytes);
return new Binary(bytes);
return message.getSerializedForm();
}
private static Map<String, Object> getMapFromMessageBody(ServerJMSMapMessage message) throws JMSException {

View File

@ -16,10 +16,12 @@
*/
package org.apache.activemq.artemis.protocol.amqp.converter.jms;
import javax.jms.JMSException;
import javax.jms.ObjectMessage;
import java.io.Serializable;
import javax.jms.JMSException;
import javax.jms.ObjectMessage;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ICoreMessage;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.qpid.proton.amqp.Binary;
@ -62,9 +64,10 @@ public class ServerJMSObjectMessage extends ServerJMSMessage implements ObjectMe
@Override
public void decode() throws Exception {
super.decode();
int size = getInnerMessage().getBodyBuffer().readInt();
ActiveMQBuffer buffer = getInnerMessage().getDataBuffer();
int size = buffer.readInt();
byte[] bytes = new byte[size];
getInnerMessage().getBodyBuffer().readBytes(bytes);
buffer.readBytes(bytes);
payload = new Binary(bytes);
}
}

View File

@ -16,14 +16,21 @@
*/
package org.apache.activemq.artemis.tests.integration.amqp;
import java.nio.charset.StandardCharsets;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import javax.jms.BytesMessage;
import javax.jms.Connection;
import javax.jms.ConnectionFactory;
import javax.jms.JMSException;
import javax.jms.Message;
import javax.jms.MessageConsumer;
import javax.jms.MessageProducer;
import javax.jms.ObjectMessage;
import javax.jms.Queue;
import javax.jms.Session;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import javax.jms.TextMessage;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.core.server.ActiveMQServer;
@ -35,6 +42,7 @@ import org.apache.activemq.transport.amqp.client.AmqpMessage;
import org.apache.activemq.transport.amqp.client.AmqpReceiver;
import org.apache.activemq.transport.amqp.client.AmqpSender;
import org.apache.activemq.transport.amqp.client.AmqpSession;
import org.apache.qpid.jms.JmsConnectionFactory;
import org.apache.qpid.proton.amqp.messaging.Data;
import org.apache.qpid.proton.message.impl.MessageImpl;
import org.junit.Assert;
@ -56,14 +64,11 @@ public class AmqpLargeMessageTest extends AmqpClientTestSupport {
protected void createAddressAndQueues(ActiveMQServer server) throws Exception {
}
@Override
protected void addAdditionalAcceptors(ActiveMQServer server) throws Exception {
//server.getConfiguration().addAcceptorConfiguration("tcp", "tcp://localhost:5445");
server.getConfiguration().addAcceptorConfiguration("tcp", "tcp://localhost:61616");
}
@Test(timeout = 60000)
public void testSendAMQPReceiveCore() throws Exception {
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
@ -85,7 +90,6 @@ public class AmqpLargeMessageTest extends AmqpClientTestSupport {
}
}
@Test(timeout = 60000)
public void testSendAMQPReceiveOpenWire() throws Exception {
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
@ -177,6 +181,120 @@ public class AmqpLargeMessageTest extends AmqpClientTestSupport {
}
}
@Test(timeout = 60000)
public void testSendAMQPReceiveAMQPViaJMSObjectMessage() throws Exception {
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
String testQueueName = "ConnectionFrameSize";
int nMsgs = 1;
ConnectionFactory factory = new JmsConnectionFactory("amqp://localhost:61616");
sendObjectMessages(nMsgs, new JmsConnectionFactory("amqp://localhost:61616"));
int count = getMessageCount(server.getPostOffice(), testQueueName);
assertEquals(nMsgs, count);
receiveJMS(nMsgs, factory);
}
@Test(timeout = 60000)
public void testSendAMQPReceiveAMQPViaJMSText() throws Exception {
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
String testQueueName = "ConnectionFrameSize";
int nMsgs = 1;
ConnectionFactory factory = new JmsConnectionFactory("amqp://localhost:61616");
sendTextMessages(nMsgs, new JmsConnectionFactory("amqp://localhost:61616"));
int count = getMessageCount(server.getPostOffice(), testQueueName);
assertEquals(nMsgs, count);
receiveJMS(nMsgs, factory);
}
@Test(timeout = 60000)
public void testSendAMQPReceiveAMQPViaJMSBytes() throws Exception {
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
String testQueueName = "ConnectionFrameSize";
int nMsgs = 1;
ConnectionFactory factory = new JmsConnectionFactory("amqp://localhost:61616");
sendBytesMessages(nMsgs, new JmsConnectionFactory("amqp://localhost:61616"));
int count = getMessageCount(server.getPostOffice(), testQueueName);
assertEquals(nMsgs, count);
receiveJMS(nMsgs, factory);
}
private void sendObjectMessages(int nMsgs, ConnectionFactory factory) throws Exception {
try (Connection connection = factory.createConnection()) {
Session session = connection.createSession();
Queue queue = session.createQueue(testQueueName);
MessageProducer producer = session.createProducer(queue);
ObjectMessage msg = session.createObjectMessage();
StringBuilder builder = new StringBuilder();
for (int i = 0; i < PAYLOAD; ++i) {
builder.append("A");
}
msg.setObject(builder.toString());
for (int i = 0; i < nMsgs; ++i) {
msg.setIntProperty("i", (Integer) i);
producer.send(msg);
}
}
}
private void sendTextMessages(int nMsgs, ConnectionFactory factory) throws Exception {
try (Connection connection = factory.createConnection()) {
Session session = connection.createSession();
Queue queue = session.createQueue(testQueueName);
MessageProducer producer = session.createProducer(queue);
TextMessage msg = session.createTextMessage();
StringBuilder builder = new StringBuilder();
for (int i = 0; i < PAYLOAD; ++i) {
builder.append("A");
}
msg.setText(builder.toString());
for (int i = 0; i < nMsgs; ++i) {
msg.setIntProperty("i", (Integer) i);
producer.send(msg);
}
}
}
private void sendBytesMessages(int nMsgs, ConnectionFactory factory) throws Exception {
try (Connection connection = factory.createConnection()) {
Session session = connection.createSession();
Queue queue = session.createQueue(testQueueName);
MessageProducer producer = session.createProducer(queue);
BytesMessage msg = session.createBytesMessage();
StringBuilder builder = new StringBuilder();
for (int i = 0; i < PAYLOAD; ++i) {
builder.append("A");
}
msg.writeBytes(builder.toString().getBytes(StandardCharsets.UTF_8));
for (int i = 0; i < nMsgs; ++i) {
msg.setIntProperty("i", (Integer) i);
producer.send(msg);
}
}
}
private AmqpMessage createAmqpMessage(byte value, int payloadSize) {
AmqpMessage message = new AmqpMessage();
byte[] payload = new byte[payloadSize];