This closes #1064
This commit is contained in:
commit
a353da0caf
|
@ -34,6 +34,7 @@ import org.apache.activemq.artemis.api.core.SimpleString;
|
|||
import org.apache.activemq.artemis.cli.Artemis;
|
||||
import org.apache.activemq.artemis.cli.commands.ActionContext;
|
||||
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessagePersister;
|
||||
import org.apache.activemq.artemis.core.paging.PagedMessage;
|
||||
import org.apache.activemq.artemis.core.paging.PagingManager;
|
||||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
|
@ -50,16 +51,22 @@ import org.apache.activemq.artemis.core.persistence.impl.journal.JournalRecordId
|
|||
import org.apache.activemq.artemis.core.persistence.impl.journal.codec.CursorAckRecordEncoding;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.journal.codec.PageUpdateTXEncoding;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.nullpm.NullStorageManager;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.CoreProtocolManagerFactory;
|
||||
import org.apache.activemq.artemis.core.server.impl.FileLockNodeManager;
|
||||
import org.apache.activemq.artemis.core.settings.HierarchicalRepository;
|
||||
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
|
||||
import org.apache.activemq.artemis.core.settings.impl.HierarchicalObjectRepository;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.MessagePersister;
|
||||
import org.apache.activemq.artemis.utils.ActiveMQThreadFactory;
|
||||
import org.apache.activemq.artemis.utils.ExecutorFactory;
|
||||
|
||||
@Command(name = "print", description = "Print data records information (WARNING: don't use while a production server is running)")
|
||||
public class PrintData extends OptionalLocking {
|
||||
|
||||
static {
|
||||
MessagePersister.registerPersister(CoreProtocolManagerFactory.ID, CoreMessagePersister.getInstance());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object execute(ActionContext context) throws Exception {
|
||||
super.execute(context);
|
||||
|
|
|
@ -16,6 +16,9 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.cli.commands.tools;
|
||||
|
||||
import javax.xml.stream.XMLOutputFactory;
|
||||
import javax.xml.stream.XMLStreamException;
|
||||
import javax.xml.stream.XMLStreamWriter;
|
||||
import java.io.File;
|
||||
import java.io.OutputStream;
|
||||
import java.lang.reflect.InvocationHandler;
|
||||
|
@ -33,14 +36,13 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
import javax.xml.stream.XMLOutputFactory;
|
||||
import javax.xml.stream.XMLStreamException;
|
||||
import javax.xml.stream.XMLStreamWriter;
|
||||
|
||||
import io.airlift.airline.Command;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.cli.commands.ActionContext;
|
||||
import org.apache.activemq.artemis.core.config.Configuration;
|
||||
|
@ -50,7 +52,7 @@ import org.apache.activemq.artemis.core.journal.PreparedTransactionInfo;
|
|||
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
||||
import org.apache.activemq.artemis.core.journal.TransactionFailureCallback;
|
||||
import org.apache.activemq.artemis.core.journal.impl.JournalImpl;
|
||||
import org.apache.activemq.artemis.core.message.BodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.paging.PagedMessage;
|
||||
import org.apache.activemq.artemis.core.paging.PagingManager;
|
||||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
|
@ -74,8 +76,6 @@ import org.apache.activemq.artemis.core.persistence.impl.journal.codec.Persisten
|
|||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.JournalType;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.settings.HierarchicalRepository;
|
||||
import org.apache.activemq.artemis.core.settings.impl.AddressSettings;
|
||||
import org.apache.activemq.artemis.core.settings.impl.HierarchicalObjectRepository;
|
||||
|
@ -83,8 +83,6 @@ import org.apache.activemq.artemis.utils.ActiveMQThreadFactory;
|
|||
import org.apache.activemq.artemis.utils.ExecutorFactory;
|
||||
import org.apache.activemq.artemis.utils.OrderedExecutorFactory;
|
||||
|
||||
import io.airlift.airline.Command;
|
||||
|
||||
@Command(name = "exp", description = "Export all message-data using an XML that could be interpreted by any system.")
|
||||
public final class XmlDataExporter extends OptionalLocking {
|
||||
|
||||
|
@ -220,7 +218,9 @@ public final class XmlDataExporter extends OptionalLocking {
|
|||
|
||||
Object o = DescribeJournal.newObjectEncoding(info, storageManager);
|
||||
if (info.getUserRecordType() == JournalRecordIds.ADD_MESSAGE) {
|
||||
messages.put(info.id, ((MessageDescribe) o).getMsg());
|
||||
messages.put(info.id, ((MessageDescribe) o).getMsg().toCore());
|
||||
} else if (info.getUserRecordType() == JournalRecordIds.ADD_MESSAGE_PROTOCOL) {
|
||||
messages.put(info.id, ((MessageDescribe) o).getMsg().toCore());
|
||||
} else if (info.getUserRecordType() == JournalRecordIds.ADD_LARGE_MESSAGE) {
|
||||
messages.put(info.id, ((MessageDescribe) o).getMsg());
|
||||
} else if (info.getUserRecordType() == JournalRecordIds.ADD_REF) {
|
||||
|
@ -361,13 +361,13 @@ public final class XmlDataExporter extends OptionalLocking {
|
|||
xmlWriter.writeEndElement(); // end BINDINGS_PARENT
|
||||
}
|
||||
|
||||
private void printAllMessagesAsXML() throws XMLStreamException {
|
||||
private void printAllMessagesAsXML() throws Exception {
|
||||
xmlWriter.writeStartElement(XmlDataConstants.MESSAGES_PARENT);
|
||||
|
||||
// Order here is important. We must process the messages from the journal before we process those from the page
|
||||
// files in order to get the messages in the right order.
|
||||
for (Map.Entry<Long, Message> messageMapEntry : messages.entrySet()) {
|
||||
printSingleMessageAsXML((ServerMessage) messageMapEntry.getValue(), extractQueueNames(messageRefs.get(messageMapEntry.getKey())));
|
||||
printSingleMessageAsXML(messageMapEntry.getValue().toCore(), extractQueueNames(messageRefs.get(messageMapEntry.getKey())));
|
||||
}
|
||||
|
||||
printPagedMessagesAsXML();
|
||||
|
@ -439,7 +439,7 @@ public final class XmlDataExporter extends OptionalLocking {
|
|||
}
|
||||
|
||||
if (queueNames.size() > 0 && (message.getTransactionID() == -1 || pgTXs.contains(message.getTransactionID()))) {
|
||||
printSingleMessageAsXML(message.getMessage(), queueNames);
|
||||
printSingleMessageAsXML(message.getMessage().toCore(), queueNames);
|
||||
}
|
||||
|
||||
messageId++;
|
||||
|
@ -456,20 +456,20 @@ public final class XmlDataExporter extends OptionalLocking {
|
|||
}
|
||||
}
|
||||
|
||||
private void printSingleMessageAsXML(ServerMessage message, List<String> queues) throws XMLStreamException {
|
||||
private void printSingleMessageAsXML(ICoreMessage message, List<String> queues) throws Exception {
|
||||
xmlWriter.writeStartElement(XmlDataConstants.MESSAGES_CHILD);
|
||||
printMessageAttributes(message);
|
||||
printMessageProperties(message);
|
||||
printMessageQueues(queues);
|
||||
printMessageBody(message);
|
||||
printMessageBody(message.toCore());
|
||||
xmlWriter.writeEndElement(); // end MESSAGES_CHILD
|
||||
messagesPrinted++;
|
||||
}
|
||||
|
||||
private void printMessageBody(ServerMessage message) throws XMLStreamException {
|
||||
private void printMessageBody(Message message) throws Exception {
|
||||
xmlWriter.writeStartElement(XmlDataConstants.MESSAGE_BODY);
|
||||
|
||||
if (message.isLargeMessage()) {
|
||||
if (message.toCore().isLargeMessage()) {
|
||||
printLargeMessageBody((LargeServerMessage) message);
|
||||
} else {
|
||||
xmlWriter.writeCData(XmlDataExporterUtil.encodeMessageBody(message));
|
||||
|
@ -479,10 +479,10 @@ public final class XmlDataExporter extends OptionalLocking {
|
|||
|
||||
private void printLargeMessageBody(LargeServerMessage message) throws XMLStreamException {
|
||||
xmlWriter.writeAttribute(XmlDataConstants.MESSAGE_IS_LARGE, Boolean.TRUE.toString());
|
||||
BodyEncoder encoder = null;
|
||||
LargeBodyEncoder encoder = null;
|
||||
|
||||
try {
|
||||
encoder = message.getBodyEncoder();
|
||||
encoder = message.toCore().getBodyEncoder();
|
||||
encoder.open();
|
||||
long totalBytesWritten = 0;
|
||||
Long bufferSize;
|
||||
|
@ -522,7 +522,7 @@ public final class XmlDataExporter extends OptionalLocking {
|
|||
xmlWriter.writeEndElement(); // end QUEUES_PARENT
|
||||
}
|
||||
|
||||
private void printMessageProperties(ServerMessage message) throws XMLStreamException {
|
||||
private void printMessageProperties(Message message) throws XMLStreamException {
|
||||
xmlWriter.writeStartElement(XmlDataConstants.PROPERTIES_PARENT);
|
||||
for (SimpleString key : message.getPropertyNames()) {
|
||||
Object value = message.getObjectProperty(key);
|
||||
|
@ -539,7 +539,7 @@ public final class XmlDataExporter extends OptionalLocking {
|
|||
xmlWriter.writeEndElement(); // end PROPERTIES_PARENT
|
||||
}
|
||||
|
||||
private void printMessageAttributes(ServerMessage message) throws XMLStreamException {
|
||||
private void printMessageAttributes(ICoreMessage message) throws XMLStreamException {
|
||||
xmlWriter.writeAttribute(XmlDataConstants.MESSAGE_ID, Long.toString(message.getMessageID()));
|
||||
xmlWriter.writeAttribute(XmlDataConstants.MESSAGE_PRIORITY, Byte.toString(message.getPriority()));
|
||||
xmlWriter.writeAttribute(XmlDataConstants.MESSAGE_EXPIRATION, Long.toString(message.getExpiration()));
|
||||
|
|
|
@ -17,10 +17,9 @@
|
|||
package org.apache.activemq.artemis.cli.commands.tools;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.utils.Base64;
|
||||
|
||||
/**
|
||||
|
@ -92,12 +91,12 @@ public class XmlDataExporterUtil {
|
|||
* @param message
|
||||
* @return
|
||||
*/
|
||||
public static String encodeMessageBody(final ServerMessage message) {
|
||||
public static String encodeMessageBody(final Message message) throws Exception {
|
||||
Preconditions.checkNotNull(message, "ServerMessage can not be null");
|
||||
|
||||
int size = message.getEndOfBodyPosition() - message.getBodyBuffer().readerIndex();
|
||||
byte[] buffer = new byte[size];
|
||||
message.getBodyBuffer().readBytes(buffer);
|
||||
ActiveMQBuffer byteBuffer = message.toCore().getReadOnlyBodyBuffer();
|
||||
byte[] buffer = new byte[byteBuffer.writerIndex()];
|
||||
byteBuffer.readBytes(buffer);
|
||||
|
||||
return XmlDataExporterUtil.encode(buffer);
|
||||
}
|
||||
|
|
|
@ -45,7 +45,9 @@ import java.util.UUID;
|
|||
|
||||
import io.airlift.airline.Command;
|
||||
import io.airlift.airline.Option;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.api.core.TransportConfiguration;
|
||||
import org.apache.activemq.artemis.api.core.client.ActiveMQClient;
|
||||
|
@ -59,11 +61,9 @@ import org.apache.activemq.artemis.api.core.management.ManagementHelper;
|
|||
import org.apache.activemq.artemis.api.core.management.ResourceNames;
|
||||
import org.apache.activemq.artemis.cli.commands.ActionAbstract;
|
||||
import org.apache.activemq.artemis.cli.commands.ActionContext;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageImpl;
|
||||
import org.apache.activemq.artemis.core.remoting.impl.netty.NettyConnectorFactory;
|
||||
import org.apache.activemq.artemis.core.remoting.impl.netty.TransportConstants;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.utils.Base64;
|
||||
import org.apache.activemq.artemis.utils.ClassloadingUtil;
|
||||
import org.apache.activemq.artemis.utils.ListUtil;
|
||||
|
@ -298,7 +298,7 @@ public final class XmlDataImporter extends ActionAbstract {
|
|||
switch (eventType) {
|
||||
case XMLStreamConstants.START_ELEMENT:
|
||||
if (XmlDataConstants.MESSAGE_BODY.equals(reader.getLocalName())) {
|
||||
processMessageBody(message);
|
||||
processMessageBody(message.toCore());
|
||||
} else if (XmlDataConstants.PROPERTIES_CHILD.equals(reader.getLocalName())) {
|
||||
processMessageProperties(message);
|
||||
} else if (XmlDataConstants.QUEUES_CHILD.equals(reader.getLocalName())) {
|
||||
|
@ -387,7 +387,7 @@ public final class XmlDataImporter extends ActionAbstract {
|
|||
logger.debug(logMessage);
|
||||
}
|
||||
|
||||
message.putBytesProperty(MessageImpl.HDR_ROUTE_TO_IDS, buffer.array());
|
||||
message.putBytesProperty(Message.HDR_ROUTE_TO_IDS, buffer.array());
|
||||
try (ClientProducer producer = session.createProducer(destination)) {
|
||||
producer.send(message);
|
||||
}
|
||||
|
@ -469,7 +469,7 @@ public final class XmlDataImporter extends ActionAbstract {
|
|||
}
|
||||
}
|
||||
|
||||
private void processMessageBody(final Message message) throws XMLStreamException, IOException {
|
||||
private void processMessageBody(final ICoreMessage message) throws XMLStreamException, IOException {
|
||||
boolean isLarge = false;
|
||||
|
||||
for (int i = 0; i < reader.getAttributeCount(); i++) {
|
||||
|
|
|
@ -0,0 +1,22 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
public interface Closeable {
|
||||
void close(boolean failed);
|
||||
}
|
|
@ -1065,6 +1065,19 @@ public interface ActiveMQBuffer extends DataInput {
|
|||
*/
|
||||
void writeBytes(ByteBuffer src);
|
||||
|
||||
|
||||
/**
|
||||
* Transfers the specified source buffer's data to this buffer starting at
|
||||
* the current {@code writerIndex} until the source buffer's position
|
||||
* reaches its limit, and increases the {@code writerIndex} by the
|
||||
* number of the transferred bytes.
|
||||
*
|
||||
* @param src The source buffer
|
||||
* @throws IndexOutOfBoundsException if {@code src.remaining()} is greater than
|
||||
* {@code this.writableBytes}
|
||||
*/
|
||||
void writeBytes(ByteBuf src, int srcIndex, int length);
|
||||
|
||||
/**
|
||||
* Returns a copy of this buffer's readable bytes. Modifying the content
|
||||
* of the returned buffer or this buffer does not affect each other at all.
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.activemq.artemis.api.core;
|
|||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
|
||||
|
@ -75,6 +76,20 @@ public final class ActiveMQBuffers {
|
|||
return buff;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an ActiveMQBuffer wrapping an underlying ByteBuf
|
||||
*
|
||||
* The position on this buffer won't affect the position on the inner buffer
|
||||
*
|
||||
* @param underlying the underlying NIO ByteBuffer
|
||||
* @return an ActiveMQBuffer wrapping the underlying NIO ByteBuffer
|
||||
*/
|
||||
public static ActiveMQBuffer wrappedBuffer(final ByteBuf underlying) {
|
||||
ActiveMQBuffer buff = new ChannelBufferWrapper(underlying.duplicate());
|
||||
|
||||
return buff;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an ActiveMQBuffer wrapping an underlying byte array
|
||||
*
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.io.Serializable;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
|
||||
/**
|
||||
|
@ -134,6 +135,39 @@ public final class SimpleString implements CharSequence, Serializable, Comparabl
|
|||
}
|
||||
|
||||
|
||||
public static SimpleString readNullableSimpleString(ByteBuf buffer) {
|
||||
int b = buffer.readByte();
|
||||
if (b == DataConstants.NULL) {
|
||||
return null;
|
||||
}
|
||||
return readSimpleString(buffer);
|
||||
}
|
||||
|
||||
|
||||
public static SimpleString readSimpleString(ByteBuf buffer) {
|
||||
int len = buffer.readInt();
|
||||
byte[] data = new byte[len];
|
||||
buffer.readBytes(data);
|
||||
return new SimpleString(data);
|
||||
}
|
||||
|
||||
public static void writeNullableSimpleString(ByteBuf buffer, SimpleString val) {
|
||||
if (val == null) {
|
||||
buffer.writeByte(DataConstants.NULL);
|
||||
} else {
|
||||
buffer.writeByte(DataConstants.NOT_NULL);
|
||||
writeSimpleString(buffer, val);
|
||||
}
|
||||
}
|
||||
|
||||
public static void writeSimpleString(ByteBuf buffer, SimpleString val) {
|
||||
byte[] data = val.getData();
|
||||
buffer.writeInt(data.length);
|
||||
buffer.writeBytes(data);
|
||||
}
|
||||
|
||||
|
||||
|
||||
public SimpleString subSeq(final int start, final int end) {
|
||||
int len = data.length >> 1;
|
||||
|
||||
|
|
|
@ -66,11 +66,7 @@ public class ChannelBufferWrapper implements ActiveMQBuffer {
|
|||
|
||||
@Override
|
||||
public SimpleString readNullableSimpleString() {
|
||||
int b = buffer.readByte();
|
||||
if (b == DataConstants.NULL) {
|
||||
return null;
|
||||
}
|
||||
return readSimpleStringInternal();
|
||||
return SimpleString.readNullableSimpleString(buffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -84,14 +80,7 @@ public class ChannelBufferWrapper implements ActiveMQBuffer {
|
|||
|
||||
@Override
|
||||
public SimpleString readSimpleString() {
|
||||
return readSimpleStringInternal();
|
||||
}
|
||||
|
||||
private SimpleString readSimpleStringInternal() {
|
||||
int len = buffer.readInt();
|
||||
byte[] data = new byte[len];
|
||||
buffer.readBytes(data);
|
||||
return new SimpleString(data);
|
||||
return SimpleString.readSimpleString(buffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -111,10 +100,21 @@ public class ChannelBufferWrapper implements ActiveMQBuffer {
|
|||
} else if (len < 0xfff) {
|
||||
return readUTF();
|
||||
} else {
|
||||
return readSimpleStringInternal().toString();
|
||||
return SimpleString.readSimpleString(buffer).toString();
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeNullableString(String val) {
|
||||
UTF8Util.writeNullableString(buffer, val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeUTF(String utf) {
|
||||
UTF8Util.saveUTF(buffer, utf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String readUTF() {
|
||||
return UTF8Util.readUTF(this);
|
||||
|
@ -127,62 +127,17 @@ public class ChannelBufferWrapper implements ActiveMQBuffer {
|
|||
|
||||
@Override
|
||||
public void writeNullableSimpleString(final SimpleString val) {
|
||||
if (val == null) {
|
||||
buffer.writeByte(DataConstants.NULL);
|
||||
} else {
|
||||
buffer.writeByte(DataConstants.NOT_NULL);
|
||||
writeSimpleStringInternal(val);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeNullableString(final String val) {
|
||||
if (val == null) {
|
||||
buffer.writeByte(DataConstants.NULL);
|
||||
} else {
|
||||
buffer.writeByte(DataConstants.NOT_NULL);
|
||||
writeStringInternal(val);
|
||||
}
|
||||
SimpleString.writeNullableSimpleString(buffer, val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeSimpleString(final SimpleString val) {
|
||||
writeSimpleStringInternal(val);
|
||||
}
|
||||
|
||||
private void writeSimpleStringInternal(final SimpleString val) {
|
||||
byte[] data = val.getData();
|
||||
buffer.writeInt(data.length);
|
||||
buffer.writeBytes(data);
|
||||
SimpleString.writeSimpleString(buffer, val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeString(final String val) {
|
||||
writeStringInternal(val);
|
||||
}
|
||||
|
||||
private void writeStringInternal(final String val) {
|
||||
int length = val.length();
|
||||
|
||||
buffer.writeInt(length);
|
||||
|
||||
if (length < 9) {
|
||||
// If very small it's more performant to store char by char
|
||||
for (int i = 0; i < val.length(); i++) {
|
||||
buffer.writeShort((short) val.charAt(i));
|
||||
}
|
||||
} else if (length < 0xfff) {
|
||||
// Store as UTF - this is quicker than char by char for most strings
|
||||
writeUTF(val);
|
||||
} else {
|
||||
// Store as SimpleString, since can't store utf > 0xffff in length
|
||||
writeSimpleStringInternal(new SimpleString(val));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeUTF(final String utf) {
|
||||
UTF8Util.saveUTF(this, utf);
|
||||
UTF8Util.writeString(buffer, val);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -575,6 +530,11 @@ public class ChannelBufferWrapper implements ActiveMQBuffer {
|
|||
buffer.writeBytes(src);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(ByteBuf src, int srcIndex, int length) {
|
||||
buffer.writeBytes(src, srcIndex, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(final ActiveMQBuffer src, final int srcIndex, final int length) {
|
||||
buffer.writeBytes(src.byteBuf(), srcIndex, length);
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/*
|
||||
/**
|
||||
* 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.
|
||||
|
@ -6,7 +6,7 @@
|
|||
* (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
|
||||
* 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,
|
||||
|
@ -14,17 +14,17 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
@Deprecated
|
||||
public class AMQPMessageTypes {
|
||||
package org.apache.activemq.artemis.core.persistence;
|
||||
|
||||
// TODO - Remove in future release as these are no longer used by the
|
||||
// inbound JMS Transformer.
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
|
||||
public static final String AMQP_TYPE_KEY = "amqp:type";
|
||||
public interface Persister<T extends Object> {
|
||||
|
||||
public static final String AMQP_SEQUENCE = "amqp:sequence";
|
||||
int getEncodeSize(T record);
|
||||
|
||||
void encode(ActiveMQBuffer buffer, T record);
|
||||
|
||||
T decode(ActiveMQBuffer buffer, T record);
|
||||
|
||||
public static final String AMQP_LIST = "amqp:list";
|
||||
}
|
|
@ -101,6 +101,14 @@ public class ByteUtil {
|
|||
}
|
||||
|
||||
public static String bytesToHex(byte[] bytes, int groupSize) {
|
||||
if (bytes == null) {
|
||||
return "NULL";
|
||||
}
|
||||
|
||||
if (bytes.length == 0) {
|
||||
return "[]";
|
||||
}
|
||||
|
||||
char[] hexChars = new char[bytes.length * 2 + numberOfGroups(bytes, groupSize)];
|
||||
int outPos = 0;
|
||||
for (int j = 0; j < bytes.length; j++) {
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQPropertyConversionException;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.logs.ActiveMQUtilBundle;
|
||||
|
@ -47,7 +47,6 @@ import static org.apache.activemq.artemis.utils.DataConstants.STRING;
|
|||
* This implementation follows section 3.5.4 of the <i>Java Message Service</i> specification
|
||||
* (Version 1.1 April 12, 2002).
|
||||
* <p>
|
||||
* TODO - should have typed property getters and do conversions herein
|
||||
*/
|
||||
public final class TypedProperties {
|
||||
|
||||
|
@ -62,6 +61,13 @@ public final class TypedProperties {
|
|||
public TypedProperties() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the number of properites
|
||||
* */
|
||||
public int size() {
|
||||
return properties.size();
|
||||
}
|
||||
|
||||
public int getMemoryOffset() {
|
||||
// The estimate is basically the encode size + 2 object references for each entry in the map
|
||||
// Note we don't include the attributes or anything else since they already included in the memory estimate
|
||||
|
@ -321,7 +327,7 @@ public final class TypedProperties {
|
|||
}
|
||||
}
|
||||
|
||||
public synchronized void decode(final ActiveMQBuffer buffer) {
|
||||
public synchronized void decode(final ByteBuf buffer) {
|
||||
byte b = buffer.readByte();
|
||||
|
||||
if (b == DataConstants.NULL) {
|
||||
|
@ -406,7 +412,7 @@ public final class TypedProperties {
|
|||
}
|
||||
}
|
||||
|
||||
public synchronized void encode(final ActiveMQBuffer buffer) {
|
||||
public synchronized void encode(final ByteBuf buffer) {
|
||||
if (properties == null) {
|
||||
buffer.writeByte(DataConstants.NULL);
|
||||
} else {
|
||||
|
@ -547,7 +553,7 @@ public final class TypedProperties {
|
|||
|
||||
abstract Object getValue();
|
||||
|
||||
abstract void write(ActiveMQBuffer buffer);
|
||||
abstract void write(ByteBuf buffer);
|
||||
|
||||
abstract int encodeSize();
|
||||
|
||||
|
@ -568,7 +574,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.NULL);
|
||||
}
|
||||
|
||||
|
@ -587,7 +593,7 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private BooleanValue(final ActiveMQBuffer buffer) {
|
||||
private BooleanValue(final ByteBuf buffer) {
|
||||
val = buffer.readBoolean();
|
||||
}
|
||||
|
||||
|
@ -597,7 +603,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.BOOLEAN);
|
||||
buffer.writeBoolean(val);
|
||||
}
|
||||
|
@ -617,7 +623,7 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private ByteValue(final ActiveMQBuffer buffer) {
|
||||
private ByteValue(final ByteBuf buffer) {
|
||||
val = buffer.readByte();
|
||||
}
|
||||
|
||||
|
@ -627,7 +633,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.BYTE);
|
||||
buffer.writeByte(val);
|
||||
}
|
||||
|
@ -646,7 +652,7 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private BytesValue(final ActiveMQBuffer buffer) {
|
||||
private BytesValue(final ByteBuf buffer) {
|
||||
int len = buffer.readInt();
|
||||
val = new byte[len];
|
||||
buffer.readBytes(val);
|
||||
|
@ -658,7 +664,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.BYTES);
|
||||
buffer.writeInt(val.length);
|
||||
buffer.writeBytes(val);
|
||||
|
@ -679,7 +685,7 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private ShortValue(final ActiveMQBuffer buffer) {
|
||||
private ShortValue(final ByteBuf buffer) {
|
||||
val = buffer.readShort();
|
||||
}
|
||||
|
||||
|
@ -689,7 +695,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.SHORT);
|
||||
buffer.writeShort(val);
|
||||
}
|
||||
|
@ -708,7 +714,7 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private IntValue(final ActiveMQBuffer buffer) {
|
||||
private IntValue(final ByteBuf buffer) {
|
||||
val = buffer.readInt();
|
||||
}
|
||||
|
||||
|
@ -718,7 +724,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.INT);
|
||||
buffer.writeInt(val);
|
||||
}
|
||||
|
@ -737,7 +743,7 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private LongValue(final ActiveMQBuffer buffer) {
|
||||
private LongValue(final ByteBuf buffer) {
|
||||
val = buffer.readLong();
|
||||
}
|
||||
|
||||
|
@ -747,7 +753,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.LONG);
|
||||
buffer.writeLong(val);
|
||||
}
|
||||
|
@ -766,7 +772,7 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private FloatValue(final ActiveMQBuffer buffer) {
|
||||
private FloatValue(final ByteBuf buffer) {
|
||||
val = Float.intBitsToFloat(buffer.readInt());
|
||||
}
|
||||
|
||||
|
@ -776,7 +782,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.FLOAT);
|
||||
buffer.writeInt(Float.floatToIntBits(val));
|
||||
}
|
||||
|
@ -796,7 +802,7 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private DoubleValue(final ActiveMQBuffer buffer) {
|
||||
private DoubleValue(final ByteBuf buffer) {
|
||||
val = Double.longBitsToDouble(buffer.readLong());
|
||||
}
|
||||
|
||||
|
@ -806,7 +812,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.DOUBLE);
|
||||
buffer.writeLong(Double.doubleToLongBits(val));
|
||||
}
|
||||
|
@ -825,7 +831,7 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private CharValue(final ActiveMQBuffer buffer) {
|
||||
private CharValue(final ByteBuf buffer) {
|
||||
val = (char) buffer.readShort();
|
||||
}
|
||||
|
||||
|
@ -835,7 +841,7 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.CHAR);
|
||||
buffer.writeShort((short) val);
|
||||
}
|
||||
|
@ -854,8 +860,8 @@ public final class TypedProperties {
|
|||
this.val = val;
|
||||
}
|
||||
|
||||
private StringValue(final ActiveMQBuffer buffer) {
|
||||
val = buffer.readSimpleString();
|
||||
private StringValue(final ByteBuf buffer) {
|
||||
val = SimpleString.readSimpleString(buffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -864,9 +870,9 @@ public final class TypedProperties {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(final ActiveMQBuffer buffer) {
|
||||
public void write(final ByteBuf buffer) {
|
||||
buffer.writeByte(DataConstants.STRING);
|
||||
buffer.writeSimpleString(val);
|
||||
SimpleString.writeSimpleString(buffer, val);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -18,7 +18,9 @@ package org.apache.activemq.artemis.utils;
|
|||
|
||||
import java.lang.ref.SoftReference;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.logs.ActiveMQUtilBundle;
|
||||
import org.apache.activemq.artemis.logs.ActiveMQUtilLogger;
|
||||
|
||||
|
@ -29,15 +31,43 @@ import org.apache.activemq.artemis.logs.ActiveMQUtilLogger;
|
|||
*/
|
||||
public final class UTF8Util {
|
||||
|
||||
private UTF8Util() {
|
||||
// utility class
|
||||
}
|
||||
|
||||
private static final boolean isTrace = ActiveMQUtilLogger.LOGGER.isTraceEnabled();
|
||||
|
||||
private static final ThreadLocal<SoftReference<StringUtilBuffer>> currenBuffer = new ThreadLocal<>();
|
||||
|
||||
public static void saveUTF(final ActiveMQBuffer out, final String str) {
|
||||
private UTF8Util() {
|
||||
// utility class
|
||||
}
|
||||
public static void writeNullableString(ByteBuf buffer, final String val) {
|
||||
if (val == null) {
|
||||
buffer.writeByte(DataConstants.NULL);
|
||||
} else {
|
||||
buffer.writeByte(DataConstants.NOT_NULL);
|
||||
writeString(buffer, val);
|
||||
}
|
||||
}
|
||||
|
||||
public static void writeString(final ByteBuf buffer, final String val) {
|
||||
int length = val.length();
|
||||
|
||||
buffer.writeInt(length);
|
||||
|
||||
if (length < 9) {
|
||||
// If very small it's more performant to store char by char
|
||||
for (int i = 0; i < val.length(); i++) {
|
||||
buffer.writeShort((short) val.charAt(i));
|
||||
}
|
||||
} else if (length < 0xfff) {
|
||||
// Store as UTF - this is quicker than char by char for most strings
|
||||
saveUTF(buffer, val);
|
||||
} else {
|
||||
// Store as SimpleString, since can't store utf > 0xffff in length
|
||||
SimpleString.writeSimpleString(buffer, new SimpleString(val));
|
||||
}
|
||||
}
|
||||
|
||||
public static void saveUTF(final ByteBuf out, final String str) {
|
||||
StringUtilBuffer buffer = UTF8Util.getThreadLocalBuffer();
|
||||
|
||||
if (str.length() > 0xffff) {
|
||||
|
|
|
@ -187,12 +187,12 @@ public class TypedPropertiesTest {
|
|||
props.putSimpleStringProperty(keyToRemove, RandomUtil.randomSimpleString());
|
||||
|
||||
ActiveMQBuffer buffer = ActiveMQBuffers.dynamicBuffer(1024);
|
||||
props.encode(buffer);
|
||||
props.encode(buffer.byteBuf());
|
||||
|
||||
Assert.assertEquals(props.getEncodeSize(), buffer.writerIndex());
|
||||
|
||||
TypedProperties decodedProps = new TypedProperties();
|
||||
decodedProps.decode(buffer);
|
||||
decodedProps.decode(buffer.byteBuf());
|
||||
|
||||
TypedPropertiesTest.assertEqualsTypeProperties(props, decodedProps);
|
||||
|
||||
|
@ -200,7 +200,7 @@ public class TypedPropertiesTest {
|
|||
|
||||
// After removing a property, you should still be able to encode the Property
|
||||
props.removeProperty(keyToRemove);
|
||||
props.encode(buffer);
|
||||
props.encode(buffer.byteBuf());
|
||||
|
||||
Assert.assertEquals(props.getEncodeSize(), buffer.writerIndex());
|
||||
}
|
||||
|
@ -210,12 +210,12 @@ public class TypedPropertiesTest {
|
|||
TypedProperties emptyProps = new TypedProperties();
|
||||
|
||||
ActiveMQBuffer buffer = ActiveMQBuffers.dynamicBuffer(1024);
|
||||
emptyProps.encode(buffer);
|
||||
emptyProps.encode(buffer.byteBuf());
|
||||
|
||||
Assert.assertEquals(props.getEncodeSize(), buffer.writerIndex());
|
||||
|
||||
TypedProperties decodedProps = new TypedProperties();
|
||||
decodedProps.decode(buffer);
|
||||
decodedProps.decode(buffer.byteBuf());
|
||||
|
||||
TypedPropertiesTest.assertEqualsTypeProperties(emptyProps, decodedProps);
|
||||
}
|
||||
|
|
|
@ -262,12 +262,6 @@ public final class ActiveMQDefaultConfiguration {
|
|||
// The minimal number of data files before we can start compacting
|
||||
private static int DEFAULT_JOURNAL_COMPACT_MIN_FILES = 10;
|
||||
|
||||
// XXX Only meant to be used by project developers
|
||||
private static int DEFAULT_JOURNAL_PERF_BLAST_PAGES = -1;
|
||||
|
||||
// XXX Only meant to be used by project developers
|
||||
private static boolean DEFAULT_RUN_SYNC_SPEED_TEST = false;
|
||||
|
||||
// Interval to log server specific information (e.g. memory usage etc)
|
||||
private static long DEFAULT_SERVER_DUMP_INTERVAL = -1;
|
||||
|
||||
|
@ -800,20 +794,6 @@ public final class ActiveMQDefaultConfiguration {
|
|||
return DEFAULT_JOURNAL_COMPACT_MIN_FILES;
|
||||
}
|
||||
|
||||
/**
|
||||
* XXX Only meant to be used by project developers
|
||||
*/
|
||||
public static int getDefaultJournalPerfBlastPages() {
|
||||
return DEFAULT_JOURNAL_PERF_BLAST_PAGES;
|
||||
}
|
||||
|
||||
/**
|
||||
* XXX Only meant to be used by project developers
|
||||
*/
|
||||
public static boolean isDefaultRunSyncSpeedTest() {
|
||||
return DEFAULT_RUN_SYNC_SPEED_TEST;
|
||||
}
|
||||
|
||||
/**
|
||||
* Interval to log server specific information (e.g. memory usage etc)
|
||||
*/
|
||||
|
|
|
@ -0,0 +1,90 @@
|
|||
/**
|
||||
* 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.api.core;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
|
||||
/**
|
||||
* This interface is only to determine the API of methods required for Core Messages
|
||||
*/
|
||||
public interface ICoreMessage extends Message {
|
||||
|
||||
LargeBodyEncoder getBodyEncoder() throws ActiveMQException;
|
||||
|
||||
int getHeadersAndPropertiesEncodeSize();
|
||||
|
||||
@Override
|
||||
InputStream getBodyInputStream();
|
||||
|
||||
/** Returns a new Buffer slicing the current Body. */
|
||||
ActiveMQBuffer getReadOnlyBodyBuffer();
|
||||
|
||||
/** Return the type of the message */
|
||||
@Override
|
||||
byte getType();
|
||||
|
||||
/** the type of the message */
|
||||
@Override
|
||||
CoreMessage setType(byte type);
|
||||
|
||||
/**
|
||||
* We are really interested if this is a LargeServerMessage.
|
||||
* @return
|
||||
*/
|
||||
boolean isServerMessage();
|
||||
|
||||
/**
|
||||
* The body used for this message.
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
ActiveMQBuffer getBodyBuffer();
|
||||
|
||||
int getEndOfBodyPosition();
|
||||
|
||||
|
||||
/** Used on large messages treatment */
|
||||
void copyHeadersAndProperties(final Message msg);
|
||||
|
||||
/**
|
||||
* @return Returns the message in Map form, useful when encoding to JSON
|
||||
*/
|
||||
@Override
|
||||
default Map<String, Object> toMap() {
|
||||
Map map = toPropertyMap();
|
||||
map.put("messageID", getMessageID());
|
||||
Object userID = getUserID();
|
||||
if (getUserID() != null) {
|
||||
map.put("userID", "ID:" + userID.toString());
|
||||
}
|
||||
|
||||
map.put("address", getAddress());
|
||||
map.put("type", getType());
|
||||
map.put("durable", isDurable());
|
||||
map.put("expiration", getExpiration());
|
||||
map.put("timestamp", getTimestamp());
|
||||
map.put("priority", (int)getPriority());
|
||||
|
||||
return map;
|
||||
}
|
||||
|
||||
}
|
|
@ -16,10 +16,13 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.api.core;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.activemq.artemis.utils.UUID;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
|
||||
/**
|
||||
* A Message is a routable instance that has a payload.
|
||||
|
@ -48,9 +51,41 @@ import org.apache.activemq.artemis.utils.UUID;
|
|||
* <p>
|
||||
* If conversion is not allowed (for example calling {@code getFloatProperty} on a property set a
|
||||
* {@code boolean}), a {@link ActiveMQPropertyConversionException} will be thrown.
|
||||
*
|
||||
*
|
||||
* User cases that will be covered by Message
|
||||
*
|
||||
* Receiving a buffer:
|
||||
*
|
||||
* Message encode = new CoreMessage(); // or any other implementation
|
||||
* encode.receiveBuffer(buffer);
|
||||
*
|
||||
*
|
||||
* Sending to a buffer:
|
||||
*
|
||||
* Message encode;
|
||||
* size = encode.getEncodeSize();
|
||||
* encode.encodeDirectly(bufferOutput);
|
||||
*
|
||||
*/
|
||||
public interface Message {
|
||||
|
||||
// This is an estimate of how much memory a Message takes up, exclusing body and properties
|
||||
// Note, it is only an estimate, it's not possible to be entirely sure with Java
|
||||
// This figure is calculated using the test utilities in org.apache.activemq.tests.unit.util.sizeof
|
||||
// The value is somewhat higher on 64 bit architectures, probably due to different alignment
|
||||
int memoryOffset = 352;
|
||||
|
||||
|
||||
SimpleString HDR_ROUTE_TO_IDS = new SimpleString("_AMQ_ROUTE_TO");
|
||||
|
||||
SimpleString HDR_SCALEDOWN_TO_IDS = new SimpleString("_AMQ_SCALEDOWN_TO");
|
||||
|
||||
SimpleString HDR_ROUTE_TO_ACK_IDS = new SimpleString("_AMQ_ACK_ROUTE_TO");
|
||||
|
||||
// used by the bridges to set duplicates
|
||||
SimpleString HDR_BRIDGE_DUPLICATE_ID = new SimpleString("_AMQ_BRIDGE_DUP");
|
||||
|
||||
/**
|
||||
* the actual time the message was expired.
|
||||
* * *
|
||||
|
@ -129,6 +164,91 @@ public interface Message {
|
|||
|
||||
byte STREAM_TYPE = 6;
|
||||
|
||||
|
||||
default SimpleString getDeliveryAnnotationPropertyString(SimpleString property) {
|
||||
Object obj = getDeliveryAnnotationProperty(property);
|
||||
if (obj instanceof SimpleString) {
|
||||
return (SimpleString)obj;
|
||||
} else {
|
||||
return SimpleString.toSimpleString(obj.toString());
|
||||
}
|
||||
}
|
||||
|
||||
default void cleanupInternalProperties() {
|
||||
// only on core
|
||||
}
|
||||
|
||||
RoutingType getRouteType();
|
||||
|
||||
boolean containsDeliveryAnnotationProperty(SimpleString property);
|
||||
|
||||
/**
|
||||
* @deprecated do not use this, use through ICoreMessage or ClientMessage
|
||||
*/
|
||||
@Deprecated
|
||||
default InputStream getBodyInputStream() {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated do not use this, use through ICoreMessage or ClientMessage
|
||||
*/
|
||||
@Deprecated
|
||||
default ActiveMQBuffer getBodyBuffer() {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated do not use this, use through ICoreMessage or ClientMessage
|
||||
*/
|
||||
@Deprecated
|
||||
default byte getType() {
|
||||
return (byte)0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated do not use this, use through ICoreMessage or ClientMessage
|
||||
*/
|
||||
@Deprecated
|
||||
default Message setType(byte type) {
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
void messageChanged();
|
||||
|
||||
/** Used to calculate what is the delivery time.
|
||||
* Return null if not scheduled. */
|
||||
Long getScheduledDeliveryTime();
|
||||
|
||||
default Message setScheduledDeliveryTime(Long time) {
|
||||
return this;
|
||||
}
|
||||
|
||||
/** Context can be used by the application server to inject extra control, like a protocol specific on the server.
|
||||
* There is only one per Object, use it wisely!
|
||||
*
|
||||
* Note: the intent of this was to replace PageStore reference on Message, but it will be later increased by adidn a ServerPojo
|
||||
* */
|
||||
RefCountMessageListener getContext();
|
||||
|
||||
SimpleString getReplyTo();
|
||||
|
||||
Message setReplyTo(SimpleString address);
|
||||
|
||||
Message setContext(RefCountMessageListener context);
|
||||
|
||||
/** The buffer will belong to this message, until release is called. */
|
||||
Message setBuffer(ByteBuf buffer);
|
||||
|
||||
ByteBuf getBuffer();
|
||||
|
||||
/** It will generate a new instance of the message encode, being a deep copy, new properties, new everything */
|
||||
Message copy();
|
||||
|
||||
/** It will generate a new instance of the message encode, being a deep copy, new properties, new everything */
|
||||
Message copy(long newID);
|
||||
|
||||
/**
|
||||
* Returns the messageID.
|
||||
* <br>
|
||||
|
@ -136,39 +256,45 @@ public interface Message {
|
|||
*/
|
||||
long getMessageID();
|
||||
|
||||
Message setMessageID(long id);
|
||||
|
||||
default boolean isLargeMessage() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the expiration time of this message.
|
||||
*/
|
||||
long getExpiration();
|
||||
|
||||
/**
|
||||
* Sets the expiration of this message.
|
||||
*
|
||||
* @param expiration expiration time
|
||||
*/
|
||||
Message setExpiration(long expiration);
|
||||
|
||||
/**
|
||||
* Returns whether this message is expired or not.
|
||||
*/
|
||||
default boolean isExpired() {
|
||||
if (getExpiration() == 0) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return System.currentTimeMillis() - getExpiration() >= 0;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns the userID - this is an optional user specified UUID that can be set to identify the message
|
||||
* and will be passed around with the message
|
||||
*
|
||||
* @return the user id
|
||||
*/
|
||||
UUID getUserID();
|
||||
Object getUserID();
|
||||
|
||||
/**
|
||||
* Sets the user ID
|
||||
*
|
||||
* @param userID
|
||||
*/
|
||||
Message setUserID(UUID userID);
|
||||
|
||||
/**
|
||||
* Returns the address this message is sent to.
|
||||
*/
|
||||
SimpleString getAddress();
|
||||
|
||||
/**
|
||||
* Sets the address to send this message to.
|
||||
*
|
||||
* @param address address to send the message to
|
||||
*/
|
||||
Message setAddress(SimpleString address);
|
||||
|
||||
/**
|
||||
* Returns this message type.
|
||||
* <p>
|
||||
* See fields {@literal *_TYPE} for possible values.
|
||||
*/
|
||||
byte getType();
|
||||
Message setUserID(Object userID);
|
||||
|
||||
/**
|
||||
* Returns whether this message is durable or not.
|
||||
|
@ -182,36 +308,18 @@ public interface Message {
|
|||
*/
|
||||
Message setDurable(boolean durable);
|
||||
|
||||
/**
|
||||
* Returns the expiration time of this message.
|
||||
*/
|
||||
long getExpiration();
|
||||
Persister<Message> getPersister();
|
||||
|
||||
/**
|
||||
* Returns whether this message is expired or not.
|
||||
*/
|
||||
boolean isExpired();
|
||||
String getAddress();
|
||||
|
||||
/**
|
||||
* Sets the expiration of this message.
|
||||
*
|
||||
* @param expiration expiration time
|
||||
*/
|
||||
Message setExpiration(long expiration);
|
||||
Message setAddress(String address);
|
||||
|
||||
SimpleString getAddressSimpleString();
|
||||
|
||||
Message setAddress(SimpleString address);
|
||||
|
||||
/**
|
||||
* Returns the message timestamp.
|
||||
* <br>
|
||||
* The timestamp corresponds to the time this message
|
||||
* was handled by an ActiveMQ Artemis server.
|
||||
*/
|
||||
long getTimestamp();
|
||||
|
||||
/**
|
||||
* Sets the message timestamp.
|
||||
*
|
||||
* @param timestamp timestamp
|
||||
*/
|
||||
Message setTimestamp(long timestamp);
|
||||
|
||||
/**
|
||||
|
@ -230,164 +338,116 @@ public interface Message {
|
|||
*/
|
||||
Message setPriority(byte priority);
|
||||
|
||||
/**
|
||||
* Returns the size of the <em>encoded</em> message.
|
||||
*/
|
||||
int getEncodeSize();
|
||||
/** Used to receive this message from an encoded medium buffer */
|
||||
void receiveBuffer(ByteBuf buffer);
|
||||
|
||||
/** Used to send this message to an encoded medium buffer.
|
||||
* @param buffer the buffer used.
|
||||
* @param deliveryCount Some protocols (AMQP) will have this as part of the message. */
|
||||
void sendBuffer(ByteBuf buffer, int deliveryCount);
|
||||
|
||||
int getPersistSize();
|
||||
|
||||
void persist(ActiveMQBuffer targetRecord);
|
||||
|
||||
void reloadPersistence(ActiveMQBuffer record);
|
||||
|
||||
default void releaseBuffer() {
|
||||
ByteBuf buffer = getBuffer();
|
||||
if (buffer != null) {
|
||||
buffer.release();
|
||||
}
|
||||
setBuffer(null);
|
||||
}
|
||||
default void referenceOriginalMessage(final Message original, String originalQueue) {
|
||||
String queueOnMessage = original.getStringProperty(Message.HDR_ORIGINAL_QUEUE.toString());
|
||||
|
||||
if (queueOnMessage != null) {
|
||||
putStringProperty(Message.HDR_ORIGINAL_QUEUE.toString(), queueOnMessage);
|
||||
} else if (originalQueue != null) {
|
||||
putStringProperty(Message.HDR_ORIGINAL_QUEUE.toString(), originalQueue);
|
||||
}
|
||||
|
||||
if (original.containsProperty(Message.HDR_ORIG_MESSAGE_ID.toString())) {
|
||||
putStringProperty(Message.HDR_ORIGINAL_ADDRESS.toString(), original.getStringProperty(Message.HDR_ORIGINAL_ADDRESS.toString()));
|
||||
|
||||
putLongProperty(Message.HDR_ORIG_MESSAGE_ID.toString(), original.getLongProperty(Message.HDR_ORIG_MESSAGE_ID.toString()));
|
||||
} else {
|
||||
putStringProperty(Message.HDR_ORIGINAL_ADDRESS.toString(), original.getAddress());
|
||||
|
||||
putLongProperty(Message.HDR_ORIG_MESSAGE_ID.toString(), original.getMessageID());
|
||||
}
|
||||
|
||||
// reset expiry
|
||||
setExpiration(0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether this message is a <em>large message</em> or a regular message.
|
||||
* it will translate a property named HDR_DUPLICATE_DETECTION_ID.
|
||||
* @return
|
||||
*/
|
||||
boolean isLargeMessage();
|
||||
default byte[] getDuplicateIDBytes() {
|
||||
Object duplicateID = getDuplicateProperty();
|
||||
|
||||
if (duplicateID == null) {
|
||||
return null;
|
||||
} else {
|
||||
if (duplicateID instanceof SimpleString) {
|
||||
return ((SimpleString) duplicateID).getData();
|
||||
} else if (duplicateID instanceof String) {
|
||||
return new SimpleString(duplicateID.toString()).getData();
|
||||
} else {
|
||||
return (byte[]) duplicateID;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the message body as an ActiveMQBuffer
|
||||
* it will translate a property named HDR_DUPLICATE_DETECTION_ID.
|
||||
* @return
|
||||
*/
|
||||
ActiveMQBuffer getBodyBuffer();
|
||||
default Object getDuplicateProperty() {
|
||||
return getDeliveryAnnotationProperty(Message.HDR_DUPLICATE_DETECTION_ID);
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes the input byte array to the message body ActiveMQBuffer
|
||||
*/
|
||||
Message writeBodyBufferBytes(byte[] bytes);
|
||||
|
||||
/**
|
||||
* Writes the input String to the message body ActiveMQBuffer
|
||||
*/
|
||||
Message writeBodyBufferString(String string);
|
||||
|
||||
/**
|
||||
* Returns a <em>copy</em> of the message body as an ActiveMQBuffer. Any modification
|
||||
* of this buffer should not impact the underlying buffer.
|
||||
*/
|
||||
ActiveMQBuffer getBodyBufferDuplicate();
|
||||
|
||||
// Properties
|
||||
// -----------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Puts a boolean property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putBooleanProperty(SimpleString key, boolean value);
|
||||
|
||||
/**
|
||||
* @see #putBooleanProperty(SimpleString, boolean)
|
||||
*/
|
||||
Message putBooleanProperty(String key, boolean value);
|
||||
|
||||
/**
|
||||
* Puts a byte property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putByteProperty(SimpleString key, byte value);
|
||||
|
||||
/**
|
||||
* @see #putByteProperty(SimpleString, byte)
|
||||
*/
|
||||
Message putByteProperty(String key, byte value);
|
||||
|
||||
/**
|
||||
* Puts a byte[] property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putBytesProperty(SimpleString key, byte[] value);
|
||||
|
||||
/**
|
||||
* @see #putBytesProperty(SimpleString, byte[])
|
||||
*/
|
||||
Message putBytesProperty(String key, byte[] value);
|
||||
|
||||
/**
|
||||
* Puts a short property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putShortProperty(SimpleString key, short value);
|
||||
|
||||
/**
|
||||
* @see #putShortProperty(SimpleString, short)
|
||||
*/
|
||||
Message putShortProperty(String key, short value);
|
||||
|
||||
/**
|
||||
* Puts a char property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putCharProperty(SimpleString key, char value);
|
||||
|
||||
/**
|
||||
* @see #putCharProperty(SimpleString, char)
|
||||
*/
|
||||
Message putCharProperty(String key, char value);
|
||||
|
||||
/**
|
||||
* Puts an int property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putIntProperty(SimpleString key, int value);
|
||||
|
||||
/**
|
||||
* @see #putIntProperty(SimpleString, int)
|
||||
*/
|
||||
Message putIntProperty(String key, int value);
|
||||
|
||||
/**
|
||||
* Puts a long property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putLongProperty(SimpleString key, long value);
|
||||
|
||||
/**
|
||||
* @see #putLongProperty(SimpleString, long)
|
||||
*/
|
||||
Message putLongProperty(String key, long value);
|
||||
|
||||
/**
|
||||
* Puts a float property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putFloatProperty(SimpleString key, float value);
|
||||
|
||||
/**
|
||||
* @see #putFloatProperty(SimpleString, float)
|
||||
*/
|
||||
Message putFloatProperty(String key, float value);
|
||||
|
||||
/**
|
||||
* Puts a double property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putDoubleProperty(SimpleString key, double value);
|
||||
|
||||
/**
|
||||
* @see #putDoubleProperty(SimpleString, double)
|
||||
*/
|
||||
Message putDoubleProperty(String key, double value);
|
||||
|
||||
/**
|
||||
* Puts a SimpleString property in this message.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
*/
|
||||
Message putStringProperty(SimpleString key, SimpleString value);
|
||||
|
||||
|
||||
Message putBooleanProperty(SimpleString key, boolean value);
|
||||
|
||||
Message putByteProperty(SimpleString key, byte value);
|
||||
|
||||
Message putBytesProperty(SimpleString key, byte[] value);
|
||||
|
||||
Message putShortProperty(SimpleString key, short value);
|
||||
|
||||
Message putCharProperty(SimpleString key, char value);
|
||||
|
||||
Message putIntProperty(SimpleString key, int value);
|
||||
|
||||
Message putLongProperty(SimpleString key, long value);
|
||||
|
||||
Message putFloatProperty(SimpleString key, float value);
|
||||
|
||||
Message putDoubleProperty(SimpleString key, double value);
|
||||
|
||||
/**
|
||||
* Puts a String property in this message.
|
||||
|
@ -397,202 +457,127 @@ public interface Message {
|
|||
*/
|
||||
Message putStringProperty(String key, String value);
|
||||
|
||||
/**
|
||||
* Puts an Object property in this message. <br>
|
||||
* Accepted types are:
|
||||
* <ul>
|
||||
* <li>Boolean</li>
|
||||
* <li>Byte</li>
|
||||
* <li>Short</li>
|
||||
* <li>Character</li>
|
||||
* <li>Integer</li>
|
||||
* <li>Long</li>
|
||||
* <li>Float</li>
|
||||
* <li>Double</li>
|
||||
* <li>String</li>
|
||||
* <li>SimpleString</li>
|
||||
* </ul>
|
||||
* Using any other type will throw a PropertyConversionException.
|
||||
*
|
||||
* @param key property name
|
||||
* @param value property value
|
||||
* @throws ActiveMQPropertyConversionException if the value is not one of the accepted property
|
||||
* types.
|
||||
*/
|
||||
Message putObjectProperty(SimpleString key, Object value) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #putObjectProperty(SimpleString, Object)
|
||||
*/
|
||||
Message putObjectProperty(String key, Object value) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Removes the property corresponding to the specified key.
|
||||
*
|
||||
* @param key property name
|
||||
* @return the value corresponding to the specified key or @{code null}
|
||||
*/
|
||||
Object removeProperty(SimpleString key);
|
||||
Message putObjectProperty(SimpleString key, Object value) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #removeProperty(SimpleString)
|
||||
*/
|
||||
Object removeProperty(String key);
|
||||
|
||||
/**
|
||||
* Returns {@code true} if this message contains a property with the given key, {@code false} else.
|
||||
*
|
||||
* @param key property name
|
||||
*/
|
||||
boolean containsProperty(SimpleString key);
|
||||
|
||||
/**
|
||||
* @see #containsProperty(SimpleString)
|
||||
*/
|
||||
boolean containsProperty(String key);
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as a Boolean.
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to a Boolean
|
||||
*/
|
||||
Boolean getBooleanProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #getBooleanProperty(SimpleString)
|
||||
*/
|
||||
Boolean getBooleanProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as a Byte.
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to a Byte
|
||||
*/
|
||||
Byte getByteProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #getByteProperty(SimpleString)
|
||||
*/
|
||||
Byte getByteProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as a Double.
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to a Double
|
||||
*/
|
||||
Double getDoubleProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #getDoubleProperty(SimpleString)
|
||||
*/
|
||||
Double getDoubleProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as an Integer.
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to an Integer
|
||||
*/
|
||||
Integer getIntProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #getIntProperty(SimpleString)
|
||||
*/
|
||||
Integer getIntProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as a Long.
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to a Long
|
||||
*/
|
||||
Long getLongProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #getLongProperty(SimpleString)
|
||||
*/
|
||||
Long getLongProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key
|
||||
*/
|
||||
Object getObjectProperty(SimpleString key);
|
||||
|
||||
/**
|
||||
* @see #getBooleanProperty(SimpleString)
|
||||
*/
|
||||
Object getObjectProperty(String key);
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as a Short.
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to a Short
|
||||
*/
|
||||
Short getShortProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #getShortProperty(SimpleString)
|
||||
*/
|
||||
Short getShortProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as a Float.
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to a Float
|
||||
*/
|
||||
Float getFloatProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #getFloatProperty(SimpleString)
|
||||
*/
|
||||
Float getFloatProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as a String.
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to a String
|
||||
*/
|
||||
String getStringProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #getStringProperty(SimpleString)
|
||||
*/
|
||||
String getStringProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as a SimpleString.
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to a SimpleString
|
||||
*/
|
||||
SimpleString getSimpleStringProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* @see #getSimpleStringProperty(SimpleString)
|
||||
*/
|
||||
SimpleString getSimpleStringProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
/**
|
||||
* Returns the property corresponding to the specified key as a byte[].
|
||||
*
|
||||
* @throws ActiveMQPropertyConversionException if the value can not be converted to a byte[]
|
||||
*/
|
||||
byte[] getBytesProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
Object removeProperty(SimpleString key);
|
||||
|
||||
boolean containsProperty(SimpleString key);
|
||||
|
||||
Boolean getBooleanProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
Byte getByteProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
Double getDoubleProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
Integer getIntProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
Long getLongProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
Object getObjectProperty(SimpleString key);
|
||||
|
||||
Object removeDeliveryAnnoationProperty(SimpleString key);
|
||||
|
||||
Object getDeliveryAnnotationProperty(SimpleString key);
|
||||
|
||||
Short getShortProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
Float getFloatProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
String getStringProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
SimpleString getSimpleStringProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
byte[] getBytesProperty(SimpleString key) throws ActiveMQPropertyConversionException;
|
||||
|
||||
Message putStringProperty(SimpleString key, SimpleString value);
|
||||
|
||||
/**
|
||||
* @see #getBytesProperty(SimpleString)
|
||||
* Returns the size of the <em>encoded</em> message.
|
||||
*/
|
||||
byte[] getBytesProperty(String key) throws ActiveMQPropertyConversionException;
|
||||
int getEncodeSize();
|
||||
|
||||
/**
|
||||
* Returns all the names of the properties for this message.
|
||||
*/
|
||||
Set<SimpleString> getPropertyNames();
|
||||
|
||||
|
||||
|
||||
int getRefCount();
|
||||
|
||||
int incrementRefCount() throws Exception;
|
||||
|
||||
int decrementRefCount() throws Exception;
|
||||
|
||||
int incrementDurableRefCount();
|
||||
|
||||
int decrementDurableRefCount();
|
||||
|
||||
/**
|
||||
* @return Returns the message in Map form, useful when encoding to JSON
|
||||
*/
|
||||
Map<String, Object> toMap();
|
||||
default Map<String, Object> toMap() {
|
||||
Map map = toPropertyMap();
|
||||
map.put("messageID", getMessageID());
|
||||
Object userID = getUserID();
|
||||
if (getUserID() != null) {
|
||||
map.put("userID", "ID:" + userID.toString());
|
||||
}
|
||||
|
||||
map.put("address", getAddress());
|
||||
map.put("durable", isDurable());
|
||||
map.put("expiration", getExpiration());
|
||||
map.put("timestamp", getTimestamp());
|
||||
map.put("priority", (int)getPriority());
|
||||
|
||||
return map;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Returns the message properties in Map form, useful when encoding to JSON
|
||||
*/
|
||||
Map<String, Object> toPropertyMap();
|
||||
default Map<String, Object> toPropertyMap() {
|
||||
Map map = new HashMap<>();
|
||||
for (SimpleString name : getPropertyNames()) {
|
||||
map.put(name.toString(), getObjectProperty(name.toString()));
|
||||
}
|
||||
return map;
|
||||
}
|
||||
|
||||
|
||||
/** This should make you convert your message into Core format. */
|
||||
ICoreMessage toCore();
|
||||
|
||||
int getMemoryEstimate();
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,81 @@
|
|||
/**
|
||||
* 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.api.core;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public abstract class RefCountMessage implements Message {
|
||||
|
||||
private final AtomicInteger durableRefCount = new AtomicInteger();
|
||||
|
||||
private final AtomicInteger refCount = new AtomicInteger();
|
||||
|
||||
private RefCountMessageListener context;
|
||||
|
||||
@Override
|
||||
public Message setContext(RefCountMessageListener context) {
|
||||
this.context = context;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RefCountMessageListener getContext() {
|
||||
return context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getRefCount() {
|
||||
return refCount.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementRefCount() throws Exception {
|
||||
int count = refCount.incrementAndGet();
|
||||
if (context != null) {
|
||||
context.nonDurableUp(this, count);
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementDurableRefCount() {
|
||||
int count = durableRefCount.incrementAndGet();
|
||||
if (context != null) {
|
||||
context.durableUp(this, count);
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementDurableRefCount() {
|
||||
int count = durableRefCount.decrementAndGet();
|
||||
if (context != null) {
|
||||
context.durableDown(this, count);
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementRefCount() throws Exception {
|
||||
int count = refCount.decrementAndGet();
|
||||
if (context != null) {
|
||||
context.nonDurableDown(this, count);
|
||||
}
|
||||
return count;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,31 @@
|
|||
/**
|
||||
* 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.api.core;
|
||||
|
||||
/** If {@link Message#getContext()} != null and is implementing this interface.
|
||||
* These methods will be called during refCount operations */
|
||||
public interface RefCountMessageListener {
|
||||
|
||||
void durableUp(Message message, int durableCount);
|
||||
|
||||
void durableDown(Message message, int durableCount);
|
||||
|
||||
void nonDurableUp(Message message, int nonDurableCoun);
|
||||
|
||||
void nonDurableDown(Message message, int nonDurableCoun);
|
||||
}
|
|
@ -19,14 +19,15 @@ package org.apache.activemq.artemis.api.core.client;
|
|||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
|
||||
/**
|
||||
* A ClientMessage represents a message sent and/or received by ActiveMQ Artemis.
|
||||
*/
|
||||
public interface ClientMessage extends Message {
|
||||
public interface ClientMessage extends ICoreMessage {
|
||||
|
||||
/**
|
||||
* Returns the number of times this message was delivered.
|
||||
|
@ -123,135 +124,141 @@ public interface ClientMessage extends Message {
|
|||
ClientMessage setBodyInputStream(InputStream bodyInputStream);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Return the bodyInputStream for large messages
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
InputStream getBodyInputStream();
|
||||
|
||||
/**
|
||||
* The buffer to write the body.
|
||||
* @return
|
||||
*/
|
||||
@Override
|
||||
ActiveMQBuffer getBodyBuffer();
|
||||
|
||||
/**
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putBooleanProperty(SimpleString key, boolean value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putBooleanProperty(String key, boolean value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putByteProperty(SimpleString key, byte value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putByteProperty(String key, byte value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putBytesProperty(SimpleString key, byte[] value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putBytesProperty(String key, byte[] value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putShortProperty(SimpleString key, short value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putShortProperty(String key, short value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putCharProperty(SimpleString key, char value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putCharProperty(String key, char value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putIntProperty(SimpleString key, int value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putIntProperty(String key, int value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putLongProperty(SimpleString key, long value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putLongProperty(String key, long value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putFloatProperty(SimpleString key, float value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putFloatProperty(String key, float value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putDoubleProperty(SimpleString key, double value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putDoubleProperty(String key, double value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putStringProperty(SimpleString key, SimpleString value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage putStringProperty(String key, String value);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage writeBodyBufferBytes(byte[] bytes);
|
||||
|
||||
/**
|
||||
* Overridden from {@link Message} to enable fluent API
|
||||
* Overridden from {@link org.apache.activemq.artemis.api.core.Message} to enable fluent API
|
||||
*/
|
||||
@Override
|
||||
ClientMessage writeBodyBufferString(String string);
|
||||
|
||||
}
|
||||
|
|
|
@ -18,9 +18,11 @@ package org.apache.activemq.artemis.api.core.management;
|
|||
|
||||
import javax.json.JsonArray;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.JsonUtil;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
|
||||
/**
|
||||
* Helper class to use ActiveMQ Artemis Core messages to manage server resources.
|
||||
|
@ -86,7 +88,7 @@ public final class ManagementHelper {
|
|||
* @param attribute the name of the attribute
|
||||
* @see ResourceNames
|
||||
*/
|
||||
public static void putAttribute(final Message message, final String resourceName, final String attribute) {
|
||||
public static void putAttribute(final ICoreMessage message, final String resourceName, final String attribute) {
|
||||
message.putStringProperty(ManagementHelper.HDR_RESOURCE_NAME, new SimpleString(resourceName));
|
||||
message.putStringProperty(ManagementHelper.HDR_ATTRIBUTE, new SimpleString(attribute));
|
||||
}
|
||||
|
@ -99,7 +101,7 @@ public final class ManagementHelper {
|
|||
* @param operationName the name of the operation to invoke on the resource
|
||||
* @see ResourceNames
|
||||
*/
|
||||
public static void putOperationInvocation(final Message message,
|
||||
public static void putOperationInvocation(final ICoreMessage message,
|
||||
final String resourceName,
|
||||
final String operationName) throws Exception {
|
||||
ManagementHelper.putOperationInvocation(message, resourceName, operationName, (Object[]) null);
|
||||
|
@ -114,7 +116,7 @@ public final class ManagementHelper {
|
|||
* @param parameters the parameters to use to invoke the server resource
|
||||
* @see ResourceNames
|
||||
*/
|
||||
public static void putOperationInvocation(final Message message,
|
||||
public static void putOperationInvocation(final ICoreMessage message,
|
||||
final String resourceName,
|
||||
final String operationName,
|
||||
final Object... parameters) throws Exception {
|
||||
|
@ -141,7 +143,7 @@ public final class ManagementHelper {
|
|||
* Used by ActiveMQ Artemis management service.
|
||||
*/
|
||||
public static Object[] retrieveOperationParameters(final Message message) throws Exception {
|
||||
SimpleString sstring = message.getBodyBuffer().readNullableSimpleString();
|
||||
SimpleString sstring = message.toCore().getReadOnlyBodyBuffer().readNullableSimpleString();
|
||||
String jsonString = (sstring == null) ? null : sstring.toString();
|
||||
|
||||
if (jsonString != null) {
|
||||
|
@ -170,7 +172,7 @@ public final class ManagementHelper {
|
|||
/**
|
||||
* Used by ActiveMQ Artemis management service.
|
||||
*/
|
||||
public static void storeResult(final Message message, final Object result) throws Exception {
|
||||
public static void storeResult(final CoreMessage message, final Object result) throws Exception {
|
||||
String resultString;
|
||||
|
||||
if (result != null) {
|
||||
|
@ -192,7 +194,7 @@ public final class ManagementHelper {
|
|||
* If an error occurred on the server, {@link #hasOperationSucceeded(Message)} will return {@code false}.
|
||||
* and the result will be a String corresponding to the server exception.
|
||||
*/
|
||||
public static Object[] getResults(final Message message) throws Exception {
|
||||
public static Object[] getResults(final ICoreMessage message) throws Exception {
|
||||
SimpleString sstring = message.getBodyBuffer().readNullableSimpleString();
|
||||
String jsonString = (sstring == null) ? null : sstring.toString();
|
||||
|
||||
|
@ -210,7 +212,7 @@ public final class ManagementHelper {
|
|||
* If an error occurred on the server, {@link #hasOperationSucceeded(Message)} will return {@code false}.
|
||||
* and the result will be a String corresponding to the server exception.
|
||||
*/
|
||||
public static Object getResult(final Message message) throws Exception {
|
||||
public static Object getResult(final ICoreMessage message) throws Exception {
|
||||
return getResult(message, null);
|
||||
}
|
||||
|
||||
|
@ -220,7 +222,7 @@ public final class ManagementHelper {
|
|||
* If an error occurred on the server, {@link #hasOperationSucceeded(Message)} will return {@code false}.
|
||||
* and the result will be a String corresponding to the server exception.
|
||||
*/
|
||||
public static Object getResult(final Message message, Class desiredType) throws Exception {
|
||||
public static Object getResult(final ICoreMessage message, Class desiredType) throws Exception {
|
||||
Object[] res = ManagementHelper.getResults(message);
|
||||
|
||||
if (res != null) {
|
||||
|
|
|
@ -20,18 +20,18 @@ import java.nio.ByteBuffer;
|
|||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
|
||||
/**
|
||||
* A ResetLimitWrappedActiveMQBuffer
|
||||
* TODO: Move this to commons
|
||||
*/
|
||||
public final class ResetLimitWrappedActiveMQBuffer extends ChannelBufferWrapper {
|
||||
|
||||
private final int limit;
|
||||
|
||||
private MessageInternal message;
|
||||
private Message message;
|
||||
|
||||
/**
|
||||
* We need to turn of notifications of body changes on reset on the server side when dealing with AMQP conversions,
|
||||
|
@ -39,17 +39,17 @@ public final class ResetLimitWrappedActiveMQBuffer extends ChannelBufferWrapper
|
|||
*
|
||||
* @param message
|
||||
*/
|
||||
public void setMessage(MessageInternal message) {
|
||||
public void setMessage(Message message) {
|
||||
this.message = message;
|
||||
}
|
||||
|
||||
public ResetLimitWrappedActiveMQBuffer(final int limit, final ActiveMQBuffer buffer, final MessageInternal message) {
|
||||
public ResetLimitWrappedActiveMQBuffer(final int limit, final ActiveMQBuffer buffer, final Message message) {
|
||||
// a wrapped inside a wrapper will increase the stack size.
|
||||
// we fixed this here due to some profiling testing
|
||||
this(limit, unwrap(buffer.byteBuf()).duplicate(), message);
|
||||
}
|
||||
|
||||
public ResetLimitWrappedActiveMQBuffer(final int limit, final ByteBuf buffer, final MessageInternal message) {
|
||||
public ResetLimitWrappedActiveMQBuffer(final int limit, final ByteBuf buffer, final Message message) {
|
||||
// a wrapped inside a wrapper will increase the stack size.
|
||||
// we fixed this here due to some profiling testing
|
||||
super(buffer);
|
||||
|
@ -67,7 +67,7 @@ public final class ResetLimitWrappedActiveMQBuffer extends ChannelBufferWrapper
|
|||
|
||||
private void changed() {
|
||||
if (message != null) {
|
||||
message.bodyChanged();
|
||||
message.messageChanged();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -94,8 +94,6 @@ public final class ResetLimitWrappedActiveMQBuffer extends ChannelBufferWrapper
|
|||
|
||||
@Override
|
||||
public void resetReaderIndex() {
|
||||
changed();
|
||||
|
||||
buffer.readerIndex(limit);
|
||||
}
|
||||
|
||||
|
@ -256,6 +254,14 @@ public final class ResetLimitWrappedActiveMQBuffer extends ChannelBufferWrapper
|
|||
super.writeBytes(src);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void writeBytes(final ByteBuf src, final int srcIndex, final int length) {
|
||||
changed();
|
||||
|
||||
super.writeBytes(src, srcIndex, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(final ActiveMQBuffer src, final int srcIndex, final int length) {
|
||||
changed();
|
||||
|
|
|
@ -569,7 +569,7 @@ public final class ClientConsumerImpl implements ClientConsumerInternal {
|
|||
|
||||
private void handleRegularMessage(ClientMessageInternal message) {
|
||||
if (message.getAddress() == null) {
|
||||
message.setAddressTransient(queueInfo.getAddress());
|
||||
message.setAddress(queueInfo.getAddress());
|
||||
}
|
||||
|
||||
message.onReceipt(this);
|
||||
|
@ -625,7 +625,7 @@ public final class ClientConsumerImpl implements ClientConsumerInternal {
|
|||
currentLargeMessageController.setLocal(true);
|
||||
|
||||
//sets the packet
|
||||
ActiveMQBuffer qbuff = clMessage.getBodyBuffer();
|
||||
ActiveMQBuffer qbuff = clMessage.toCore().getBodyBuffer();
|
||||
int bytesToRead = qbuff.writerIndex() - qbuff.readerIndex();
|
||||
final byte[] body = ByteUtil.getActiveArray(qbuff.readBytes(bytesToRead).toByteBuffer());
|
||||
|
||||
|
|
|
@ -59,7 +59,7 @@ public final class ClientLargeMessageImpl extends ClientMessageImpl implements C
|
|||
|
||||
@Override
|
||||
public int getEncodeSize() {
|
||||
if (bodyBuffer != null) {
|
||||
if (writableBuffer != null) {
|
||||
return super.getEncodeSize();
|
||||
} else {
|
||||
return DataConstants.SIZE_INT + DataConstants.SIZE_INT + getHeadersAndPropertiesEncodeSize();
|
||||
|
@ -93,7 +93,7 @@ public final class ClientLargeMessageImpl extends ClientMessageImpl implements C
|
|||
throw new RuntimeException(e.getMessage(), e);
|
||||
}
|
||||
|
||||
return bodyBuffer;
|
||||
return writableBuffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -108,7 +108,7 @@ public final class ClientLargeMessageImpl extends ClientMessageImpl implements C
|
|||
|
||||
@Override
|
||||
public void saveToOutputStream(final OutputStream out) throws ActiveMQException {
|
||||
if (bodyBuffer != null) {
|
||||
if (writableBuffer != null) {
|
||||
// The body was rebuilt on the client, so we need to behave as a regular message on this case
|
||||
super.saveToOutputStream(out);
|
||||
} else {
|
||||
|
@ -118,7 +118,7 @@ public final class ClientLargeMessageImpl extends ClientMessageImpl implements C
|
|||
|
||||
@Override
|
||||
public ClientLargeMessageImpl setOutputStream(final OutputStream out) throws ActiveMQException {
|
||||
if (bodyBuffer != null) {
|
||||
if (writableBuffer != null) {
|
||||
super.setOutputStream(out);
|
||||
} else {
|
||||
largeMessageController.setOutputStream(out);
|
||||
|
@ -129,7 +129,7 @@ public final class ClientLargeMessageImpl extends ClientMessageImpl implements C
|
|||
|
||||
@Override
|
||||
public boolean waitOutputStreamCompletion(final long timeMilliseconds) throws ActiveMQException {
|
||||
if (bodyBuffer != null) {
|
||||
if (writableBuffer != null) {
|
||||
return super.waitOutputStreamCompletion(timeMilliseconds);
|
||||
} else {
|
||||
return largeMessageController.waitCompletion(timeMilliseconds);
|
||||
|
@ -138,7 +138,7 @@ public final class ClientLargeMessageImpl extends ClientMessageImpl implements C
|
|||
|
||||
@Override
|
||||
public void discardBody() {
|
||||
if (bodyBuffer != null) {
|
||||
if (writableBuffer != null) {
|
||||
super.discardBody();
|
||||
} else {
|
||||
largeMessageController.discardUnusedPackets();
|
||||
|
@ -146,17 +146,17 @@ public final class ClientLargeMessageImpl extends ClientMessageImpl implements C
|
|||
}
|
||||
|
||||
private void checkBuffer() throws ActiveMQException {
|
||||
if (bodyBuffer == null) {
|
||||
if (writableBuffer == null) {
|
||||
|
||||
long bodySize = this.largeMessageSize + BODY_OFFSET;
|
||||
if (bodySize > Integer.MAX_VALUE) {
|
||||
bodySize = Integer.MAX_VALUE;
|
||||
}
|
||||
createBody((int) bodySize);
|
||||
initBuffer((int) bodySize);
|
||||
|
||||
bodyBuffer = new ResetLimitWrappedActiveMQBuffer(BODY_OFFSET, buffer, this);
|
||||
writableBuffer = new ResetLimitWrappedActiveMQBuffer(BODY_OFFSET, buffer.duplicate(), this);
|
||||
|
||||
largeMessageController.saveBuffer(new ActiveMQOutputStream(bodyBuffer));
|
||||
largeMessageController.saveBuffer(new ActiveMQOutputStream(writableBuffer));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -178,7 +178,7 @@ public final class ClientLargeMessageImpl extends ClientMessageImpl implements C
|
|||
|
||||
public void retrieveExistingData(ClientMessageInternal clMessage) {
|
||||
this.messageID = clMessage.getMessageID();
|
||||
this.address = clMessage.getAddress();
|
||||
this.address = clMessage.getAddressSimpleString();
|
||||
this.setUserID(clMessage.getUserID());
|
||||
this.setFlowControlSize(clMessage.getFlowControlSize());
|
||||
this.setDeliveryCount(clMessage.getDeliveryCount());
|
||||
|
|
|
@ -28,14 +28,16 @@ import org.apache.activemq.artemis.api.core.ActiveMQPropertyConversionException;
|
|||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.client.ActiveMQClientMessageBundle;
|
||||
import org.apache.activemq.artemis.core.message.BodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageImpl;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.reader.MessageUtil;
|
||||
import org.apache.activemq.artemis.utils.TypedProperties;
|
||||
import org.apache.activemq.artemis.utils.UUID;
|
||||
|
||||
/**
|
||||
* A ClientMessageImpl
|
||||
*/
|
||||
public class ClientMessageImpl extends MessageImpl implements ClientMessageInternal {
|
||||
public class ClientMessageImpl extends CoreMessage implements ClientMessageInternal {
|
||||
|
||||
// added this constant here so that the client package have no dependency on JMS
|
||||
public static final SimpleString REPLYTO_HEADER_NAME = MessageUtil.REPLYTO_HEADER_NAME;
|
||||
|
@ -57,6 +59,35 @@ public class ClientMessageImpl extends MessageImpl implements ClientMessageInter
|
|||
public ClientMessageImpl() {
|
||||
}
|
||||
|
||||
protected ClientMessageImpl(ClientMessageImpl other) {
|
||||
super(other);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClientMessageImpl setDurable(boolean durable) {
|
||||
super.setDurable(durable);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClientMessageImpl setExpiration(long expiration) {
|
||||
super.setExpiration(expiration);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClientMessageImpl setPriority(byte priority) {
|
||||
super.setPriority(priority);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClientMessageImpl setUserID(UUID userID) {
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* Construct messages before sending
|
||||
*/
|
||||
|
@ -66,12 +97,13 @@ public class ClientMessageImpl extends MessageImpl implements ClientMessageInter
|
|||
final long timestamp,
|
||||
final byte priority,
|
||||
final int initialMessageBufferSize) {
|
||||
super(type, durable, expiration, timestamp, priority, initialMessageBufferSize);
|
||||
this.setType(type).setExpiration(expiration).setTimestamp(timestamp).setDurable(durable).
|
||||
setPriority(priority).initBuffer(initialMessageBufferSize);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isServerMessage() {
|
||||
return false;
|
||||
public TypedProperties getProperties() {
|
||||
return this.checkProperties();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -108,6 +140,11 @@ public class ClientMessageImpl extends MessageImpl implements ClientMessageInter
|
|||
return this;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void checkCompletion() throws ActiveMQException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getFlowControlSize() {
|
||||
if (flowControlSize < 0) {
|
||||
|
@ -141,7 +178,7 @@ public class ClientMessageImpl extends MessageImpl implements ClientMessageInter
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName() + "[messageID=" + messageID + ", durable=" + durable + ", address=" + getAddress() + ",userID=" + (getUserID() != null ? getUserID() : "null") + ",properties=" + properties.toString() + "]";
|
||||
return getClass().getSimpleName() + "[messageID=" + messageID + ", durable=" + durable + ", address=" + getAddress() + ",userID=" + (getUserID() != null ? getUserID() : "null") + ",properties=" + getProperties().toString() + "]";
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -189,7 +226,7 @@ public class ClientMessageImpl extends MessageImpl implements ClientMessageInter
|
|||
}
|
||||
|
||||
@Override
|
||||
public BodyEncoder getBodyEncoder() throws ActiveMQException {
|
||||
public LargeBodyEncoder getBodyEncoder() throws ActiveMQException {
|
||||
return new DecodingContext();
|
||||
}
|
||||
|
||||
|
@ -307,15 +344,17 @@ public class ClientMessageImpl extends MessageImpl implements ClientMessageInter
|
|||
|
||||
@Override
|
||||
public ClientMessageImpl writeBodyBufferBytes(byte[] bytes) {
|
||||
return (ClientMessageImpl) super.writeBodyBufferBytes(bytes);
|
||||
getBodyBuffer().writeBytes(bytes);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClientMessageImpl writeBodyBufferString(String string) {
|
||||
return (ClientMessageImpl) super.writeBodyBufferString(string);
|
||||
getBodyBuffer().writeString(string);
|
||||
return this;
|
||||
}
|
||||
|
||||
private final class DecodingContext implements BodyEncoder {
|
||||
private final class DecodingContext implements LargeBodyEncoder {
|
||||
|
||||
private DecodingContext() {
|
||||
}
|
||||
|
@ -347,9 +386,15 @@ public class ClientMessageImpl extends MessageImpl implements ClientMessageInter
|
|||
@Override
|
||||
public int encode(final ActiveMQBuffer bufferOut, final int size) {
|
||||
byte[] bytes = new byte[size];
|
||||
getWholeBuffer().readBytes(bytes);
|
||||
buffer.readBytes(bytes);
|
||||
bufferOut.writeBytes(bytes, 0, size);
|
||||
return size;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message copy() {
|
||||
return new ClientMessageImpl(this);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.core.client.impl;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.api.core.client.ClientMessage;
|
||||
import org.apache.activemq.artemis.utils.TypedProperties;
|
||||
|
||||
|
@ -34,8 +33,6 @@ public interface ClientMessageInternal extends ClientMessage {
|
|||
*/
|
||||
void setFlowControlSize(int flowControlSize);
|
||||
|
||||
void setAddressTransient(SimpleString address);
|
||||
|
||||
void onReceipt(ClientConsumerInternal consumer);
|
||||
|
||||
/**
|
||||
|
@ -44,4 +41,5 @@ public interface ClientMessageInternal extends ClientMessage {
|
|||
void discardBody();
|
||||
|
||||
boolean isCompressed();
|
||||
|
||||
}
|
||||
|
|
|
@ -23,12 +23,12 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.api.core.client.SendAcknowledgementHandler;
|
||||
import org.apache.activemq.artemis.core.client.ActiveMQClientMessageBundle;
|
||||
import org.apache.activemq.artemis.core.message.BodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.SessionContext;
|
||||
import org.apache.activemq.artemis.utils.ActiveMQBufferInputStream;
|
||||
import org.apache.activemq.artemis.utils.DeflaterReader;
|
||||
|
@ -208,7 +208,7 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
}
|
||||
|
||||
private void doSend(SimpleString sendingAddress,
|
||||
final Message msg,
|
||||
final Message msgToSend,
|
||||
final SendAcknowledgementHandler handler,
|
||||
final boolean forceAsync) throws ActiveMQException {
|
||||
if (sendingAddress == null) {
|
||||
|
@ -217,7 +217,8 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
session.startCall();
|
||||
|
||||
try {
|
||||
MessageInternal msgI = (MessageInternal) msg;
|
||||
// In case we received message from another protocol, we first need to convert it to core as the ClientProducer only understands core
|
||||
ICoreMessage msg = msgToSend.toCore();
|
||||
|
||||
ClientProducerCredits theCredits;
|
||||
|
||||
|
@ -225,8 +226,8 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
// a note about the second check on the writerIndexSize,
|
||||
// If it's a server's message, it means this is being done through the bridge or some special consumer on the
|
||||
// server's on which case we can't' convert the message into large at the servers
|
||||
if (sessionContext.supportsLargeMessage() && (msgI.getBodyInputStream() != null || msgI.isLargeMessage() ||
|
||||
msgI.getBodyBuffer().writerIndex() > minLargeMessageSize && !msgI.isServerMessage())) {
|
||||
if (sessionContext.supportsLargeMessage() && (getBodyInputStream(msg) != null || msg.isLargeMessage() ||
|
||||
msg.getBodyBuffer().writerIndex() > minLargeMessageSize)) {
|
||||
isLarge = true;
|
||||
} else {
|
||||
isLarge = false;
|
||||
|
@ -248,27 +249,31 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
}
|
||||
|
||||
if (groupID != null) {
|
||||
msgI.putStringProperty(Message.HDR_GROUP_ID, groupID);
|
||||
msg.putStringProperty(Message.HDR_GROUP_ID, groupID);
|
||||
}
|
||||
|
||||
final boolean sendBlockingConfig = msgI.isDurable() ? blockOnDurableSend : blockOnNonDurableSend;
|
||||
final boolean sendBlockingConfig = msg.isDurable() ? blockOnDurableSend : blockOnNonDurableSend;
|
||||
final boolean forceAsyncOverride = handler != null;
|
||||
final boolean sendBlocking = sendBlockingConfig && !forceAsyncOverride;
|
||||
|
||||
session.workDone();
|
||||
|
||||
if (isLarge) {
|
||||
largeMessageSend(sendBlocking, msgI, theCredits, handler);
|
||||
largeMessageSend(sendBlocking, msg, theCredits, handler);
|
||||
} else {
|
||||
sendRegularMessage(sendingAddress, msgI, sendBlocking, theCredits, handler);
|
||||
sendRegularMessage(sendingAddress, msg, sendBlocking, theCredits, handler);
|
||||
}
|
||||
} finally {
|
||||
session.endCall();
|
||||
}
|
||||
}
|
||||
|
||||
private InputStream getBodyInputStream(ICoreMessage msgI) {
|
||||
return msgI.getBodyInputStream();
|
||||
}
|
||||
|
||||
private void sendRegularMessage(final SimpleString sendingAddress,
|
||||
final MessageInternal msgI,
|
||||
final ICoreMessage msgI,
|
||||
final boolean sendBlocking,
|
||||
final ClientProducerCredits theCredits,
|
||||
final SendAcknowledgementHandler handler) throws ActiveMQException {
|
||||
|
@ -301,7 +306,7 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
* @throws ActiveMQException
|
||||
*/
|
||||
private void largeMessageSend(final boolean sendBlocking,
|
||||
final MessageInternal msgI,
|
||||
final ICoreMessage msgI,
|
||||
final ClientProducerCredits credits,
|
||||
SendAcknowledgementHandler handler) throws ActiveMQException {
|
||||
logger.tracef("largeMessageSend::%s, Blocking=%s", msgI, sendBlocking);
|
||||
|
@ -313,22 +318,22 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
}
|
||||
|
||||
// msg.getBody() could be Null on LargeServerMessage
|
||||
if (msgI.getBodyInputStream() == null && msgI.getWholeBuffer() != null) {
|
||||
msgI.getWholeBuffer().readerIndex(0);
|
||||
if (getBodyInputStream(msgI) == null && msgI.getBuffer() != null) {
|
||||
msgI.getBuffer().readerIndex(0);
|
||||
}
|
||||
|
||||
InputStream input;
|
||||
|
||||
if (msgI.isServerMessage()) {
|
||||
largeMessageSendServer(sendBlocking, msgI, credits, handler);
|
||||
} else if ((input = msgI.getBodyInputStream()) != null) {
|
||||
} else if ((input = getBodyInputStream(msgI)) != null) {
|
||||
largeMessageSendStreamed(sendBlocking, msgI, input, credits, handler);
|
||||
} else {
|
||||
largeMessageSendBuffered(sendBlocking, msgI, credits, handler);
|
||||
}
|
||||
}
|
||||
|
||||
private void sendInitialLargeMessageHeader(MessageInternal msgI,
|
||||
private void sendInitialLargeMessageHeader(Message msgI,
|
||||
ClientProducerCredits credits) throws ActiveMQException {
|
||||
int creditsUsed = sessionContext.sendInitialChunkOnLargeMessage(msgI);
|
||||
|
||||
|
@ -348,17 +353,14 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
* @throws ActiveMQException
|
||||
*/
|
||||
private void largeMessageSendServer(final boolean sendBlocking,
|
||||
final MessageInternal msgI,
|
||||
final ICoreMessage msgI,
|
||||
final ClientProducerCredits credits,
|
||||
SendAcknowledgementHandler handler) throws ActiveMQException {
|
||||
sendInitialLargeMessageHeader(msgI, credits);
|
||||
|
||||
BodyEncoder context = msgI.getBodyEncoder();
|
||||
LargeBodyEncoder context = msgI.getBodyEncoder();
|
||||
|
||||
final long bodySize = context.getLargeBodySize();
|
||||
|
||||
final int reconnectID = sessionContext.getReconnectID();
|
||||
|
||||
context.open();
|
||||
try {
|
||||
|
||||
|
@ -392,7 +394,7 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
* @throws ActiveMQException
|
||||
*/
|
||||
private void largeMessageSendBuffered(final boolean sendBlocking,
|
||||
final MessageInternal msgI,
|
||||
final ICoreMessage msgI,
|
||||
final ClientProducerCredits credits,
|
||||
SendAcknowledgementHandler handler) throws ActiveMQException {
|
||||
msgI.getBodyBuffer().readerIndex(0);
|
||||
|
@ -407,7 +409,7 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
* @throws ActiveMQException
|
||||
*/
|
||||
private void largeMessageSendStreamed(final boolean sendBlocking,
|
||||
final MessageInternal msgI,
|
||||
final ICoreMessage msgI,
|
||||
final InputStream inputStreamParameter,
|
||||
final ClientProducerCredits credits,
|
||||
SendAcknowledgementHandler handler) throws ActiveMQException {
|
||||
|
@ -478,7 +480,7 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
|||
msgI.putLongProperty(Message.HDR_LARGE_BODY_SIZE, deflaterReader.getTotalSize());
|
||||
|
||||
msgI.getBodyBuffer().writeBytes(buff, 0, pos);
|
||||
sendRegularMessage(msgI.getAddress(), msgI, sendBlocking, credits, handler);
|
||||
sendRegularMessage(msgI.getAddressSimpleString(), msgI, sendBlocking, credits, handler);
|
||||
return;
|
||||
} else {
|
||||
if (!headerSent) {
|
||||
|
|
|
@ -512,6 +512,12 @@ final class CompressedLargeMessageControllerImpl implements LargeMessageControll
|
|||
throw new IllegalAccessError(OPERATION_NOT_SUPPORTED);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(ByteBuf src, int srcIndex, int length) {
|
||||
throw new IllegalAccessError(OPERATION_NOT_SUPPORTED);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public ByteBuffer toByteBuffer() {
|
||||
throw new IllegalAccessError(OPERATION_NOT_SUPPORTED);
|
||||
|
|
|
@ -863,6 +863,21 @@ public class LargeMessageControllerImpl implements LargeMessageController {
|
|||
throw new IllegalAccessError(LargeMessageControllerImpl.READ_ONLY_ERROR_MESSAGE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Transfers the specified source buffer's data to this buffer starting at
|
||||
* the current {@code writerIndex} until the source buffer's position
|
||||
* reaches its limit, and increases the {@code writerIndex} by the
|
||||
* number of the transferred bytes.
|
||||
*
|
||||
* @param src The source buffer
|
||||
* @throws IndexOutOfBoundsException if {@code src.remaining()} is greater than
|
||||
* {@code this.writableBytes}
|
||||
*/
|
||||
@Override
|
||||
public void writeBytes(ByteBuf src, int srcIndex, int length) {
|
||||
throw new IllegalAccessError(LargeMessageControllerImpl.READ_ONLY_ERROR_MESSAGE);
|
||||
}
|
||||
|
||||
public int writeBytes(final InputStream in, final int length) throws IOException {
|
||||
throw new IllegalAccessError(LargeMessageControllerImpl.READ_ONLY_ERROR_MESSAGE);
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQException;
|
|||
* <br>
|
||||
* Used to send large streams over the wire
|
||||
*/
|
||||
public interface BodyEncoder {
|
||||
public interface LargeBodyEncoder {
|
||||
|
||||
/**
|
||||
* This method must not be called directly by ActiveMQ Artemis clients.
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,66 @@
|
|||
/**
|
||||
* 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.message.impl;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
|
||||
public class CoreMessagePersister implements Persister<Message> {
|
||||
|
||||
public static CoreMessagePersister theInstance = new CoreMessagePersister();
|
||||
|
||||
public static CoreMessagePersister getInstance() {
|
||||
return theInstance;
|
||||
}
|
||||
|
||||
protected CoreMessagePersister() {
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int getEncodeSize(Message record) {
|
||||
return DataConstants.SIZE_BYTE + record.getPersistSize() +
|
||||
SimpleString.sizeofNullableString(record.getAddressSimpleString()) + DataConstants.SIZE_LONG;
|
||||
}
|
||||
|
||||
|
||||
/** Sub classes must add the first short as the protocol-id */
|
||||
@Override
|
||||
public void encode(ActiveMQBuffer buffer, Message record) {
|
||||
buffer.writeByte((byte)1);
|
||||
buffer.writeLong(record.getMessageID());
|
||||
buffer.writeNullableSimpleString(record.getAddressSimpleString());
|
||||
record.persist(buffer);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Message decode(ActiveMQBuffer buffer, Message record) {
|
||||
// the caller must consume the first byte already, as that will be used to decide what persister (protocol) to use
|
||||
long id = buffer.readLong();
|
||||
SimpleString address = buffer.readNullableSimpleString();
|
||||
record = new CoreMessage();
|
||||
record.reloadPersistence(buffer);
|
||||
record.setMessageID(id);
|
||||
record.setAddress(address);
|
||||
return record;
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -1,57 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.core.message.impl;
|
||||
|
||||
import java.io.InputStream;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.BodyEncoder;
|
||||
import org.apache.activemq.artemis.utils.TypedProperties;
|
||||
|
||||
public interface MessageInternal extends Message {
|
||||
|
||||
void decodeFromBuffer(ActiveMQBuffer buffer);
|
||||
|
||||
int getEndOfMessagePosition();
|
||||
|
||||
int getEndOfBodyPosition();
|
||||
|
||||
void bodyChanged();
|
||||
|
||||
boolean isServerMessage();
|
||||
|
||||
ActiveMQBuffer getEncodedBuffer();
|
||||
|
||||
int getHeadersAndPropertiesEncodeSize();
|
||||
|
||||
ActiveMQBuffer getWholeBuffer();
|
||||
|
||||
void encodeHeadersAndProperties(ActiveMQBuffer buffer);
|
||||
|
||||
void decodeHeadersAndProperties(ActiveMQBuffer buffer);
|
||||
|
||||
BodyEncoder getBodyEncoder() throws ActiveMQException;
|
||||
|
||||
InputStream getBodyInputStream();
|
||||
|
||||
void setAddressTransient(SimpleString address);
|
||||
|
||||
TypedProperties getTypedProperties();
|
||||
}
|
|
@ -31,7 +31,9 @@ import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
|
|||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.api.core.client.ClientConsumer;
|
||||
import org.apache.activemq.artemis.api.core.client.ClientSession;
|
||||
|
@ -45,7 +47,7 @@ import org.apache.activemq.artemis.core.client.impl.ClientLargeMessageInternal;
|
|||
import org.apache.activemq.artemis.core.client.impl.ClientMessageInternal;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientProducerCreditsImpl;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientSessionImpl;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.protocol.core.Channel;
|
||||
import org.apache.activemq.artemis.core.protocol.core.ChannelHandler;
|
||||
import org.apache.activemq.artemis.core.protocol.core.CommandConfirmationHandler;
|
||||
|
@ -103,7 +105,6 @@ import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionXAR
|
|||
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionXASetTimeoutMessage;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionXASetTimeoutResponseMessage;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionXAStartMessage;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.Connection;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.ReadyListener;
|
||||
|
@ -422,12 +423,12 @@ public class ActiveMQSessionContext extends SessionContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int getCreditsOnSendingFull(MessageInternal msgI) {
|
||||
public int getCreditsOnSendingFull(Message msgI) {
|
||||
return msgI.getEncodeSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendFullMessage(MessageInternal msgI,
|
||||
public void sendFullMessage(ICoreMessage msgI,
|
||||
boolean sendBlocking,
|
||||
SendAcknowledgementHandler handler,
|
||||
SimpleString defaultAddress) throws ActiveMQException {
|
||||
|
@ -441,16 +442,16 @@ public class ActiveMQSessionContext extends SessionContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendInitialChunkOnLargeMessage(MessageInternal msgI) throws ActiveMQException {
|
||||
public int sendInitialChunkOnLargeMessage(Message msgI) throws ActiveMQException {
|
||||
SessionSendLargeMessage initialChunk = new SessionSendLargeMessage(msgI);
|
||||
|
||||
sessionChannel.send(initialChunk);
|
||||
|
||||
return msgI.getHeadersAndPropertiesEncodeSize();
|
||||
return ((CoreMessage)msgI).getHeadersAndPropertiesEncodeSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int sendLargeMessageChunk(MessageInternal msgI,
|
||||
public int sendLargeMessageChunk(Message msgI,
|
||||
long messageBodySize,
|
||||
boolean sendBlocking,
|
||||
boolean lastChunk,
|
||||
|
@ -471,7 +472,7 @@ public class ActiveMQSessionContext extends SessionContext {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendServerLargeMessageChunk(MessageInternal msgI,
|
||||
public int sendServerLargeMessageChunk(Message msgI,
|
||||
long messageBodySize,
|
||||
boolean sendBlocking,
|
||||
boolean lastChunk,
|
||||
|
|
|
@ -371,6 +371,7 @@ public final class ChannelImpl implements Channel {
|
|||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("Sending blocking " + packet);
|
||||
}
|
||||
|
||||
connection.getTransportConnection().write(buffer, false, false);
|
||||
|
||||
long toWait = connection.getBlockingCallTimeout();
|
||||
|
|
|
@ -16,8 +16,11 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.core.protocol.core.impl;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
|
||||
import org.apache.activemq.artemis.core.protocol.core.Packet;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
|
@ -25,6 +28,7 @@ import org.apache.activemq.artemis.utils.DataConstants;
|
|||
public class PacketImpl implements Packet {
|
||||
// Constants -------------------------------------------------------------------------
|
||||
|
||||
|
||||
public static final int ADDRESSING_CHANGE_VERSION = 129;
|
||||
|
||||
public static final SimpleString OLD_QUEUE_PREFIX = new SimpleString("jms.queue.");
|
||||
|
@ -310,7 +314,7 @@ public class PacketImpl implements Packet {
|
|||
|
||||
@Override
|
||||
public ActiveMQBuffer encode(final RemotingConnection connection, boolean usePooled) {
|
||||
ActiveMQBuffer buffer = connection.createTransportBuffer(PacketImpl.INITIAL_PACKET_SIZE, usePooled);
|
||||
ActiveMQBuffer buffer = createPacket(connection, usePooled);
|
||||
|
||||
// The standard header fields
|
||||
|
||||
|
@ -330,6 +334,14 @@ public class PacketImpl implements Packet {
|
|||
return buffer;
|
||||
}
|
||||
|
||||
protected ActiveMQBuffer createPacket(RemotingConnection connection, boolean usePooled) {
|
||||
if (connection == null) {
|
||||
return new ChannelBufferWrapper(Unpooled.buffer(INITIAL_PACKET_SIZE));
|
||||
} else {
|
||||
return connection.createTransportBuffer(PacketImpl.INITIAL_PACKET_SIZE, usePooled);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decode(final ActiveMQBuffer buffer) {
|
||||
channelID = buffer.readLong();
|
||||
|
@ -339,6 +351,22 @@ public class PacketImpl implements Packet {
|
|||
size = buffer.readerIndex();
|
||||
}
|
||||
|
||||
protected ByteBuf copyMessageBuffer(ByteBuf buffer, int skipBytes) {
|
||||
|
||||
ByteBuf newNettyBuffer = Unpooled.buffer(buffer.capacity() - PACKET_HEADERS_SIZE - skipBytes);
|
||||
|
||||
int read = buffer.readerIndex();
|
||||
int writ = buffer.writerIndex();
|
||||
buffer.readerIndex(PACKET_HEADERS_SIZE);
|
||||
|
||||
newNettyBuffer.writeBytes(buffer, buffer.readableBytes() - skipBytes);
|
||||
buffer.setIndex( read, writ );
|
||||
newNettyBuffer.setIndex( 0, writ - PACKET_HEADERS_SIZE - skipBytes);
|
||||
|
||||
return newNettyBuffer;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int getPacketSize() {
|
||||
if (size == -1) {
|
||||
|
|
|
@ -353,6 +353,7 @@ public class RemotingConnectionImpl extends AbstractRemotingConnection implement
|
|||
}
|
||||
|
||||
dataReceived = true;
|
||||
|
||||
doBufferReceived(packet);
|
||||
|
||||
super.bufferReceived(connectionID, buffer);
|
||||
|
|
|
@ -16,22 +16,25 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.core.protocol.core.impl.wireformat;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
|
||||
public abstract class MessagePacket extends PacketImpl implements MessagePacketI {
|
||||
|
||||
protected MessageInternal message;
|
||||
protected ICoreMessage message;
|
||||
|
||||
public MessagePacket(final byte type, final MessageInternal message) {
|
||||
public MessagePacket(final byte type, final ICoreMessage message) {
|
||||
super(type);
|
||||
|
||||
this.message = message;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message getMessage() {
|
||||
public ICoreMessage getMessage() {
|
||||
return message;
|
||||
}
|
||||
|
||||
|
@ -40,4 +43,12 @@ public abstract class MessagePacket extends PacketImpl implements MessagePacketI
|
|||
return super.getParentString() + ", message=" + message;
|
||||
}
|
||||
|
||||
protected ActiveMQBuffer internalCreatePacket(int size, RemotingConnection connection, boolean usePooled) {
|
||||
if (connection == null) {
|
||||
return new ChannelBufferWrapper(Unpooled.buffer(size));
|
||||
} else {
|
||||
return connection.createTransportBuffer(size, usePooled);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,12 +17,13 @@
|
|||
package org.apache.activemq.artemis.core.protocol.core.impl.wireformat;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientLargeMessageInternal;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
|
||||
public class SessionReceiveClientLargeMessage extends SessionReceiveLargeMessage {
|
||||
|
||||
public SessionReceiveClientLargeMessage(MessageInternal message) {
|
||||
public SessionReceiveClientLargeMessage(Message message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,12 +18,12 @@ package org.apache.activemq.artemis.core.protocol.core.impl.wireformat;
|
|||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl;
|
||||
|
||||
public class SessionReceiveLargeMessage extends PacketImpl implements MessagePacketI {
|
||||
|
||||
private final MessageInternal message;
|
||||
private final Message message;
|
||||
|
||||
/**
|
||||
* Since we receive the message before the entire message was received,
|
||||
|
@ -35,13 +35,13 @@ public class SessionReceiveLargeMessage extends PacketImpl implements MessagePac
|
|||
private int deliveryCount;
|
||||
|
||||
// To be used on decoding at the client while receiving a large message
|
||||
public SessionReceiveLargeMessage(final MessageInternal message) {
|
||||
public SessionReceiveLargeMessage(final Message message) {
|
||||
super(SESS_RECEIVE_LARGE_MSG);
|
||||
this.message = message;
|
||||
}
|
||||
|
||||
public SessionReceiveLargeMessage(final long consumerID,
|
||||
final MessageInternal message,
|
||||
final Message message,
|
||||
final long largeMessageSize,
|
||||
final int deliveryCount) {
|
||||
super(SESS_RECEIVE_LARGE_MSG);
|
||||
|
@ -55,7 +55,7 @@ public class SessionReceiveLargeMessage extends PacketImpl implements MessagePac
|
|||
this.largeMessageSize = largeMessageSize;
|
||||
}
|
||||
|
||||
public MessageInternal getLargeMessage() {
|
||||
public Message getLargeMessage() {
|
||||
return message;
|
||||
}
|
||||
|
||||
|
@ -85,7 +85,7 @@ public class SessionReceiveLargeMessage extends PacketImpl implements MessagePac
|
|||
buffer.writeInt(deliveryCount);
|
||||
buffer.writeLong(largeMessageSize);
|
||||
if (message != null) {
|
||||
message.encodeHeadersAndProperties(buffer);
|
||||
((CoreMessage)message).encodeHeadersAndProperties(buffer.byteBuf());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -94,7 +94,7 @@ public class SessionReceiveLargeMessage extends PacketImpl implements MessagePac
|
|||
consumerID = buffer.readLong();
|
||||
deliveryCount = buffer.readInt();
|
||||
largeMessageSize = buffer.readLong();
|
||||
message.decodeHeadersAndProperties(buffer);
|
||||
((CoreMessage)message).decodeHeadersAndProperties(buffer.byteBuf());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -17,7 +17,8 @@
|
|||
package org.apache.activemq.artemis.core.protocol.core.impl.wireformat;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
|
||||
|
@ -30,7 +31,7 @@ public class SessionReceiveMessage extends MessagePacket {
|
|||
|
||||
private int deliveryCount;
|
||||
|
||||
public SessionReceiveMessage(final long consumerID, final MessageInternal message, final int deliveryCount) {
|
||||
public SessionReceiveMessage(final long consumerID, final ICoreMessage message, final int deliveryCount) {
|
||||
super(SESS_RECEIVE_MSG, message);
|
||||
|
||||
this.consumerID = consumerID;
|
||||
|
@ -38,7 +39,7 @@ public class SessionReceiveMessage extends MessagePacket {
|
|||
this.deliveryCount = deliveryCount;
|
||||
}
|
||||
|
||||
public SessionReceiveMessage(final MessageInternal message) {
|
||||
public SessionReceiveMessage(final CoreMessage message) {
|
||||
super(SESS_RECEIVE_MSG, message);
|
||||
}
|
||||
|
||||
|
@ -53,53 +54,28 @@ public class SessionReceiveMessage extends MessagePacket {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ActiveMQBuffer encode(final RemotingConnection connection) {
|
||||
ActiveMQBuffer buffer = message.getEncodedBuffer();
|
||||
|
||||
ActiveMQBuffer bufferWrite = connection.createTransportBuffer(buffer.writerIndex() + DataConstants.SIZE_LONG + DataConstants.SIZE_INT, true);
|
||||
bufferWrite.writeBytes(buffer, 0, buffer.capacity());
|
||||
bufferWrite.setIndex(buffer.readerIndex(), buffer.writerIndex());
|
||||
|
||||
// Sanity check
|
||||
if (bufferWrite.writerIndex() != message.getEndOfMessagePosition()) {
|
||||
throw new IllegalStateException("Wrong encode position");
|
||||
}
|
||||
|
||||
bufferWrite.writeLong(consumerID);
|
||||
bufferWrite.writeInt(deliveryCount);
|
||||
|
||||
size = bufferWrite.writerIndex();
|
||||
|
||||
// Write standard headers
|
||||
|
||||
int len = size - DataConstants.SIZE_INT;
|
||||
bufferWrite.setInt(0, len);
|
||||
bufferWrite.setByte(DataConstants.SIZE_INT, getType());
|
||||
bufferWrite.setLong(DataConstants.SIZE_INT + DataConstants.SIZE_BYTE, channelID);
|
||||
|
||||
// Position reader for reading by Netty
|
||||
bufferWrite.setIndex(0, size);
|
||||
|
||||
return bufferWrite;
|
||||
protected ActiveMQBuffer createPacket(RemotingConnection connection, boolean usePooled) {
|
||||
return internalCreatePacket(message.getEncodeSize() + PACKET_HEADERS_SIZE + DataConstants.SIZE_LONG + DataConstants.SIZE_INT, connection, usePooled);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decode(final ActiveMQBuffer buffer) {
|
||||
channelID = buffer.readLong();
|
||||
|
||||
message.decodeFromBuffer(buffer);
|
||||
|
||||
consumerID = buffer.readLong();
|
||||
|
||||
deliveryCount = buffer.readInt();
|
||||
|
||||
size = buffer.readerIndex();
|
||||
|
||||
// Need to position buffer for reading
|
||||
|
||||
buffer.setIndex(PACKET_HEADERS_SIZE + DataConstants.SIZE_INT, message.getEndOfBodyPosition());
|
||||
public void encodeRest(ActiveMQBuffer buffer) {
|
||||
message.sendBuffer(buffer.byteBuf(), deliveryCount);
|
||||
buffer.writeLong(consumerID);
|
||||
buffer.writeInt(deliveryCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decodeRest(final ActiveMQBuffer buffer) {
|
||||
// Buffer comes in after having read standard headers and positioned at Beginning of body part
|
||||
|
||||
message.receiveBuffer(copyMessageBuffer(buffer.byteBuf(), DataConstants.SIZE_LONG + DataConstants.SIZE_INT));
|
||||
|
||||
buffer.readerIndex(buffer.capacity() - DataConstants.SIZE_LONG - DataConstants.SIZE_INT);
|
||||
this.consumerID = buffer.readLong();
|
||||
this.deliveryCount = buffer.readInt();
|
||||
|
||||
}
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
|
|
|
@ -17,8 +17,8 @@
|
|||
package org.apache.activemq.artemis.core.protocol.core.impl.wireformat;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.client.SendAcknowledgementHandler;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
|
||||
/**
|
||||
* A SessionSendContinuationMessage<br>
|
||||
|
@ -28,7 +28,7 @@ public class SessionSendContinuationMessage extends SessionContinuationMessage {
|
|||
private boolean requiresResponse;
|
||||
|
||||
// Used on confirmation handling
|
||||
private MessageInternal message;
|
||||
private Message message;
|
||||
/**
|
||||
* In case, we are using a different handler than the one set on the {@link org.apache.activemq.artemis.api.core.client.ClientSession}
|
||||
* <br>
|
||||
|
@ -58,7 +58,7 @@ public class SessionSendContinuationMessage extends SessionContinuationMessage {
|
|||
* @param continues
|
||||
* @param requiresResponse
|
||||
*/
|
||||
public SessionSendContinuationMessage(final MessageInternal message,
|
||||
public SessionSendContinuationMessage(final Message message,
|
||||
final byte[] body,
|
||||
final boolean continues,
|
||||
final boolean requiresResponse,
|
||||
|
@ -87,7 +87,7 @@ public class SessionSendContinuationMessage extends SessionContinuationMessage {
|
|||
/**
|
||||
* @return the message
|
||||
*/
|
||||
public MessageInternal getMessage() {
|
||||
public Message getMessage() {
|
||||
return message;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@ package org.apache.activemq.artemis.core.protocol.core.impl.wireformat;
|
|||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl;
|
||||
|
||||
public class SessionSendLargeMessage extends PacketImpl implements MessagePacketI {
|
||||
|
@ -26,13 +26,13 @@ public class SessionSendLargeMessage extends PacketImpl implements MessagePacket
|
|||
/**
|
||||
* Used only if largeMessage
|
||||
*/
|
||||
private final MessageInternal largeMessage;
|
||||
private final Message largeMessage;
|
||||
|
||||
// Static --------------------------------------------------------
|
||||
|
||||
// Constructors --------------------------------------------------
|
||||
|
||||
public SessionSendLargeMessage(final MessageInternal largeMessage) {
|
||||
public SessionSendLargeMessage(final Message largeMessage) {
|
||||
super(SESS_SEND_LARGE);
|
||||
|
||||
this.largeMessage = largeMessage;
|
||||
|
@ -40,7 +40,7 @@ public class SessionSendLargeMessage extends PacketImpl implements MessagePacket
|
|||
|
||||
// Public --------------------------------------------------------
|
||||
|
||||
public MessageInternal getLargeMessage() {
|
||||
public Message getLargeMessage() {
|
||||
return largeMessage;
|
||||
}
|
||||
|
||||
|
@ -51,12 +51,12 @@ public class SessionSendLargeMessage extends PacketImpl implements MessagePacket
|
|||
|
||||
@Override
|
||||
public void encodeRest(final ActiveMQBuffer buffer) {
|
||||
largeMessage.encodeHeadersAndProperties(buffer);
|
||||
((CoreMessage)largeMessage).encodeHeadersAndProperties(buffer.byteBuf());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decodeRest(final ActiveMQBuffer buffer) {
|
||||
largeMessage.decodeHeadersAndProperties(buffer);
|
||||
((CoreMessage)largeMessage).decodeHeadersAndProperties(buffer.byteBuf());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -16,11 +16,12 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.core.protocol.core.impl.wireformat;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.client.SendAcknowledgementHandler;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
|
||||
public class SessionSendMessage extends MessagePacket {
|
||||
|
||||
|
@ -36,7 +37,8 @@ public class SessionSendMessage extends MessagePacket {
|
|||
*/
|
||||
private final transient SendAcknowledgementHandler handler;
|
||||
|
||||
public SessionSendMessage(final MessageInternal message,
|
||||
/** This will be using the CoreMessage because it is meant for the core-protocol */
|
||||
public SessionSendMessage(final ICoreMessage message,
|
||||
final boolean requiresResponse,
|
||||
final SendAcknowledgementHandler handler) {
|
||||
super(SESS_SEND, message);
|
||||
|
@ -44,7 +46,7 @@ public class SessionSendMessage extends MessagePacket {
|
|||
this.requiresResponse = requiresResponse;
|
||||
}
|
||||
|
||||
public SessionSendMessage(final MessageInternal message) {
|
||||
public SessionSendMessage(final CoreMessage message) {
|
||||
super(SESS_SEND, message);
|
||||
this.handler = null;
|
||||
}
|
||||
|
@ -60,53 +62,29 @@ public class SessionSendMessage extends MessagePacket {
|
|||
}
|
||||
|
||||
@Override
|
||||
public ActiveMQBuffer encode(final RemotingConnection connection) {
|
||||
ActiveMQBuffer buffer = message.getEncodedBuffer();
|
||||
protected ActiveMQBuffer createPacket(RemotingConnection connection, boolean usePooled) {
|
||||
return internalCreatePacket(message.getEncodeSize() + PACKET_HEADERS_SIZE + 1, connection, usePooled);
|
||||
}
|
||||
|
||||
ActiveMQBuffer bufferWrite;
|
||||
if (connection == null) {
|
||||
// this is for unit tests only
|
||||
bufferWrite = buffer.copy(0, buffer.capacity());
|
||||
} else {
|
||||
bufferWrite = connection.createTransportBuffer(buffer.writerIndex() + 1, true); // 1 for the requireResponse
|
||||
}
|
||||
bufferWrite.writeBytes(buffer, 0, buffer.writerIndex());
|
||||
bufferWrite.setIndex(buffer.readerIndex(), buffer.writerIndex());
|
||||
@Override
|
||||
public void encodeRest(ActiveMQBuffer buffer) {
|
||||
message.sendBuffer(buffer.byteBuf(), 0);
|
||||
buffer.writeBoolean(requiresResponse);
|
||||
|
||||
// Sanity check
|
||||
if (bufferWrite.writerIndex() != message.getEndOfMessagePosition()) {
|
||||
throw new IllegalStateException("Wrong encode position");
|
||||
}
|
||||
|
||||
bufferWrite.writeBoolean(requiresResponse);
|
||||
|
||||
size = bufferWrite.writerIndex();
|
||||
|
||||
// Write standard headers
|
||||
|
||||
int len = size - DataConstants.SIZE_INT;
|
||||
bufferWrite.setInt(0, len);
|
||||
bufferWrite.setByte(DataConstants.SIZE_INT, getType());
|
||||
bufferWrite.setLong(DataConstants.SIZE_INT + DataConstants.SIZE_BYTE, channelID);
|
||||
|
||||
// Position reader for reading by Netty
|
||||
bufferWrite.readerIndex(0);
|
||||
|
||||
return bufferWrite;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decodeRest(final ActiveMQBuffer buffer) {
|
||||
// Buffer comes in after having read standard headers and positioned at Beginning of body part
|
||||
|
||||
message.decodeFromBuffer(buffer);
|
||||
ByteBuf messageBuffer = copyMessageBuffer(buffer.byteBuf(), 1);
|
||||
message.receiveBuffer(messageBuffer);
|
||||
|
||||
int ri = buffer.readerIndex();
|
||||
buffer.readerIndex(buffer.capacity() - 1);
|
||||
|
||||
requiresResponse = buffer.readBoolean();
|
||||
|
||||
buffer.readerIndex(ri);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -26,7 +26,7 @@ public class MapMessageUtil extends MessageUtil {
|
|||
*/
|
||||
public static void writeBodyMap(ActiveMQBuffer message, TypedProperties properties) {
|
||||
message.resetWriterIndex();
|
||||
properties.encode(message);
|
||||
properties.encode(message.byteBuf());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -43,7 +43,7 @@ public class MapMessageUtil extends MessageUtil {
|
|||
*/
|
||||
public static void readBodyMap(ActiveMQBuffer message, TypedProperties map) {
|
||||
message.resetReaderIndex();
|
||||
map.decode(message);
|
||||
map.decode(message.byteBuf());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -23,7 +23,9 @@ import java.util.Set;
|
|||
import java.util.concurrent.Executor;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.api.core.client.ClientConsumer;
|
||||
import org.apache.activemq.artemis.api.core.client.ClientSession;
|
||||
|
@ -33,8 +35,6 @@ import org.apache.activemq.artemis.core.client.impl.ClientLargeMessageInternal;
|
|||
import org.apache.activemq.artemis.core.client.impl.ClientMessageInternal;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientProducerCreditsImpl;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientSessionInternal;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.activemq.artemis.utils.SimpleIDGenerator;
|
||||
|
@ -128,9 +128,9 @@ public abstract class SessionContext {
|
|||
|
||||
}
|
||||
|
||||
public abstract int getCreditsOnSendingFull(MessageInternal msgI);
|
||||
public abstract int getCreditsOnSendingFull(Message msgI);
|
||||
|
||||
public abstract void sendFullMessage(MessageInternal msgI,
|
||||
public abstract void sendFullMessage(ICoreMessage msgI,
|
||||
boolean sendBlocking,
|
||||
SendAcknowledgementHandler handler,
|
||||
SimpleString defaultAddress) throws ActiveMQException;
|
||||
|
@ -142,9 +142,9 @@ public abstract class SessionContext {
|
|||
* @return
|
||||
* @throws ActiveMQException
|
||||
*/
|
||||
public abstract int sendInitialChunkOnLargeMessage(MessageInternal msgI) throws ActiveMQException;
|
||||
public abstract int sendInitialChunkOnLargeMessage(Message msgI) throws ActiveMQException;
|
||||
|
||||
public abstract int sendLargeMessageChunk(MessageInternal msgI,
|
||||
public abstract int sendLargeMessageChunk(Message msgI,
|
||||
long messageBodySize,
|
||||
boolean sendBlocking,
|
||||
boolean lastChunk,
|
||||
|
@ -152,7 +152,7 @@ public abstract class SessionContext {
|
|||
int reconnectID,
|
||||
SendAcknowledgementHandler messageHandler) throws ActiveMQException;
|
||||
|
||||
public abstract int sendServerLargeMessageChunk(MessageInternal msgI,
|
||||
public abstract int sendServerLargeMessageChunk(Message msgI,
|
||||
long messageBodySize,
|
||||
boolean sendBlocking,
|
||||
boolean lastChunk,
|
||||
|
|
|
@ -0,0 +1,365 @@
|
|||
/**
|
||||
* 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.message;
|
||||
|
||||
import java.util.LinkedList;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientMessageImpl;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionReceiveMessage;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.wireformat.SessionSendMessage;
|
||||
import org.apache.activemq.artemis.reader.TextMessageUtil;
|
||||
import org.apache.activemq.artemis.utils.Base64;
|
||||
import org.apache.activemq.artemis.utils.ByteUtil;
|
||||
import org.apache.activemq.artemis.utils.UUID;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
public class CoreMessageTest {
|
||||
|
||||
public static final SimpleString ADDRESS = new SimpleString("this.local.address");
|
||||
public static final byte MESSAGE_TYPE = Message.TEXT_TYPE;
|
||||
public static final boolean DURABLE = true;
|
||||
public static final long EXPIRATION = 123L;
|
||||
public static final long TIMESTAMP = 321L;
|
||||
public static final byte PRIORITY = (byte) 3;
|
||||
public static final String TEXT = "hi";
|
||||
public static final String BIGGER_TEXT = "AAAAAAAAAAAAAAAAAAAAAAAAA ASDF ASDF ASF ASD ASF ASDF ASDF ASDF ASF ADSF ASDF";
|
||||
public static final String SMALLER_TEXT = "H";
|
||||
public static final UUID uuid = new UUID(UUID.TYPE_TIME_BASED, new byte[]{0, 0, 0, 0,
|
||||
0, 0, 0, 0,
|
||||
0, 0, 0, 0,
|
||||
0, 0, 0, 1});
|
||||
public static final SimpleString PROP1_NAME = new SimpleString("t1");
|
||||
public static final SimpleString PROP1_VALUE = new SimpleString("value-t1");
|
||||
|
||||
/**
|
||||
* This encode was generated by {@link #generate()}.
|
||||
* Run it manually with a right-click on the IDE to eventually update it
|
||||
* */
|
||||
// body = "hi";
|
||||
private final String STRING_ENCODE = "AAAAFgEAAAAEaABpAAAAAAAAAAAAAQAAACR0AGgAaQBzAC4AbABvAGMAYQBsAC4AYQBkAGQAcgBlAHMAcwAAAwEAAAAAAAAAewAAAAAAAAFBAwEAAAABAAAABHQAMQAKAAAAEHYAYQBsAHUAZQAtAHQAMQA=";
|
||||
|
||||
private ByteBuf BYTE_ENCODE;
|
||||
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
BYTE_ENCODE = Unpooled.wrappedBuffer(Base64.decode(STRING_ENCODE, Base64.DONT_BREAK_LINES | Base64.URL_SAFE));
|
||||
// some extra caution here, nothing else, to make sure we would get the same encoding back
|
||||
Assert.assertEquals(STRING_ENCODE, encodeString(BYTE_ENCODE.array()));
|
||||
BYTE_ENCODE.readerIndex(0).writerIndex(BYTE_ENCODE.capacity());
|
||||
}
|
||||
|
||||
/** The message is received, then sent to the other side untouched */
|
||||
@Test
|
||||
public void testPassThrough() {
|
||||
CoreMessage decodedMessage = decodeMessage();
|
||||
|
||||
Assert.assertEquals(TEXT, TextMessageUtil.readBodyText(decodedMessage.getReadOnlyBodyBuffer()).toString());
|
||||
}
|
||||
|
||||
/** The message is received, then sent to the other side untouched */
|
||||
@Test
|
||||
public void sendThroughPackets() {
|
||||
CoreMessage decodedMessage = decodeMessage();
|
||||
|
||||
int encodeSize = decodedMessage.getEncodeSize();
|
||||
Assert.assertEquals(BYTE_ENCODE.capacity(), encodeSize);
|
||||
|
||||
SessionSendMessage sendMessage = new SessionSendMessage(decodedMessage, true, null);
|
||||
sendMessage.setChannelID(777);
|
||||
|
||||
ActiveMQBuffer buffer = sendMessage.encode(null);
|
||||
|
||||
byte[] byteArray = buffer.byteBuf().array();
|
||||
System.out.println("Sending " + ByteUtil.bytesToHex(buffer.toByteBuffer().array(), 1) + ", bytes = " + byteArray.length);
|
||||
|
||||
buffer.readerIndex(5);
|
||||
|
||||
SessionSendMessage sendMessageReceivedSent = new SessionSendMessage(new CoreMessage());
|
||||
|
||||
sendMessageReceivedSent.decode(buffer);
|
||||
|
||||
Assert.assertEquals(encodeSize, sendMessageReceivedSent.getMessage().getEncodeSize());
|
||||
|
||||
Assert.assertTrue(sendMessageReceivedSent.isRequiresResponse());
|
||||
|
||||
Assert.assertEquals(TEXT, TextMessageUtil.readBodyText(sendMessageReceivedSent.getMessage().getReadOnlyBodyBuffer()).toString());
|
||||
}
|
||||
|
||||
/** The message is received, then sent to the other side untouched */
|
||||
@Test
|
||||
public void sendThroughPacketsClient() {
|
||||
CoreMessage decodedMessage = decodeMessage();
|
||||
|
||||
int encodeSize = decodedMessage.getEncodeSize();
|
||||
Assert.assertEquals(BYTE_ENCODE.capacity(), encodeSize);
|
||||
|
||||
SessionReceiveMessage sendMessage = new SessionReceiveMessage(33, decodedMessage, 7);
|
||||
sendMessage.setChannelID(777);
|
||||
|
||||
ActiveMQBuffer buffer = sendMessage.encode(null);
|
||||
|
||||
buffer.readerIndex(5);
|
||||
|
||||
SessionReceiveMessage sendMessageReceivedSent = new SessionReceiveMessage(new CoreMessage());
|
||||
|
||||
sendMessageReceivedSent.decode(buffer);
|
||||
|
||||
Assert.assertEquals(33, sendMessageReceivedSent.getConsumerID());
|
||||
|
||||
Assert.assertEquals(7, sendMessageReceivedSent.getDeliveryCount());
|
||||
|
||||
Assert.assertEquals(encodeSize, sendMessageReceivedSent.getMessage().getEncodeSize());
|
||||
|
||||
Assert.assertEquals(TEXT, TextMessageUtil.readBodyText(sendMessageReceivedSent.getMessage().getReadOnlyBodyBuffer()).toString());
|
||||
}
|
||||
|
||||
private CoreMessage decodeMessage() {
|
||||
|
||||
ByteBuf newBuffer = Unpooled.buffer(BYTE_ENCODE.capacity());
|
||||
newBuffer.writeBytes(BYTE_ENCODE, 0, BYTE_ENCODE.writerIndex());
|
||||
|
||||
CoreMessage coreMessage = internalDecode(newBuffer);
|
||||
|
||||
int encodeSize = coreMessage.getEncodeSize();
|
||||
|
||||
Assert.assertEquals(newBuffer.capacity(), encodeSize);
|
||||
|
||||
Assert.assertEquals(ADDRESS, coreMessage.getAddressSimpleString());
|
||||
|
||||
Assert.assertEquals(PROP1_VALUE.toString(), coreMessage.getStringProperty(PROP1_NAME));
|
||||
|
||||
ByteBuf destinedBuffer = Unpooled.buffer(BYTE_ENCODE.array().length);
|
||||
coreMessage.sendBuffer(destinedBuffer, 0);
|
||||
|
||||
byte[] destinedArray = destinedBuffer.array();
|
||||
byte[] sourceArray = BYTE_ENCODE.array();
|
||||
|
||||
CoreMessage newDecoded = internalDecode(Unpooled.wrappedBuffer(destinedArray));
|
||||
|
||||
Assert.assertEquals(encodeSize, newDecoded.getEncodeSize());
|
||||
|
||||
Assert.assertArrayEquals(sourceArray, destinedArray);
|
||||
|
||||
return coreMessage;
|
||||
}
|
||||
|
||||
private CoreMessage internalDecode(ByteBuf bufferOrigin) {
|
||||
CoreMessage coreMessage = new CoreMessage();
|
||||
// System.out.println("Bytes from test " + ByteUtil.bytesToHex(bufferOrigin.array(), 1));
|
||||
coreMessage.receiveBuffer(bufferOrigin);
|
||||
return coreMessage;
|
||||
}
|
||||
|
||||
/** The message is received, then sent to the other side untouched */
|
||||
@Test
|
||||
public void testChangeBodyStringSameSize() {
|
||||
testChangeBodyString(TEXT.toUpperCase());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChangeBodyBiggerString() {
|
||||
testChangeBodyString(BIGGER_TEXT);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGenerateEmpty() {
|
||||
CoreMessage empty = new CoreMessage().initBuffer(100);
|
||||
ByteBuf buffer = Unpooled.buffer(200);
|
||||
empty.sendBuffer(buffer, 0);
|
||||
|
||||
CoreMessage empty2 = new CoreMessage();
|
||||
empty2.receiveBuffer(buffer);
|
||||
|
||||
try {
|
||||
empty2.getBodyBuffer().readByte();
|
||||
Assert.fail("should throw exception");
|
||||
} catch (Exception expected) {
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSaveReceiveLimitedBytes() {
|
||||
CoreMessage empty = new CoreMessage().initBuffer(100);
|
||||
System.out.println("R " + empty.getBodyBuffer().readerIndex() + " W " + empty.getBodyBuffer().writerIndex());
|
||||
empty.getBodyBuffer().writeByte((byte)7);
|
||||
System.out.println("R " + empty.getBodyBuffer().readerIndex() + " W " + empty.getBodyBuffer().writerIndex());
|
||||
|
||||
ByteBuf buffer = Unpooled.buffer(200);
|
||||
empty.sendBuffer(buffer, 0);
|
||||
|
||||
CoreMessage empty2 = new CoreMessage();
|
||||
empty2.receiveBuffer(buffer);
|
||||
|
||||
Assert.assertEquals((byte)7, empty2.getBodyBuffer().readByte());
|
||||
|
||||
System.out.println("Readable :: " + empty2.getBodyBuffer().readerIndex() + " writer :" + empty2.getBodyBuffer().writerIndex());
|
||||
|
||||
try {
|
||||
empty2.getBodyBuffer().readByte();
|
||||
Assert.fail("should throw exception");
|
||||
} catch (Exception expected) {
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChangeBodySmallerString() {
|
||||
testChangeBodyString(SMALLER_TEXT);
|
||||
}
|
||||
|
||||
public void testChangeBodyString(String newString) {
|
||||
CoreMessage coreMessage = decodeMessage();
|
||||
|
||||
coreMessage.putStringProperty("newProperty", "newValue");
|
||||
ActiveMQBuffer legacyBuffer = coreMessage.getBodyBuffer();
|
||||
legacyBuffer.resetWriterIndex();
|
||||
legacyBuffer.clear();
|
||||
|
||||
TextMessageUtil.writeBodyText(legacyBuffer, SimpleString.toSimpleString(newString));
|
||||
|
||||
ByteBuf newbuffer = Unpooled.buffer(150000);
|
||||
|
||||
coreMessage.sendBuffer(newbuffer, 0);
|
||||
newbuffer.readerIndex(0);
|
||||
|
||||
CoreMessage newCoreMessage = new CoreMessage();
|
||||
newCoreMessage.receiveBuffer(newbuffer);
|
||||
|
||||
|
||||
SimpleString newText = TextMessageUtil.readBodyText(newCoreMessage.getReadOnlyBodyBuffer());
|
||||
|
||||
Assert.assertEquals(newString, newText.toString());
|
||||
|
||||
// coreMessage.putStringProperty()
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPassThroughMultipleThreads() throws Throwable {
|
||||
CoreMessage coreMessage = new CoreMessage();
|
||||
coreMessage.receiveBuffer(BYTE_ENCODE);
|
||||
|
||||
LinkedList<Throwable> errors = new LinkedList<>();
|
||||
|
||||
Thread[] threads = new Thread[50];
|
||||
for (int i = 0; i < threads.length; i++) {
|
||||
threads[i] = new Thread(() -> {
|
||||
try {
|
||||
for (int j = 0; j < 50; j++) {
|
||||
Assert.assertEquals(ADDRESS, coreMessage.getAddressSimpleString());
|
||||
Assert.assertEquals(PROP1_VALUE.toString(), coreMessage.getStringProperty(PROP1_NAME));
|
||||
|
||||
ByteBuf destinedBuffer = Unpooled.buffer(BYTE_ENCODE.array().length);
|
||||
coreMessage.sendBuffer(destinedBuffer, 0);
|
||||
|
||||
byte[] destinedArray = destinedBuffer.array();
|
||||
byte[] sourceArray = BYTE_ENCODE.array();
|
||||
|
||||
Assert.assertArrayEquals(sourceArray, destinedArray);
|
||||
|
||||
Assert.assertEquals(TEXT, TextMessageUtil.readBodyText(coreMessage.getReadOnlyBodyBuffer()).toString());
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
e.printStackTrace();
|
||||
errors.add(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
for (Thread t : threads) {
|
||||
t.start();
|
||||
}
|
||||
|
||||
for (Thread t : threads) {
|
||||
t.join();
|
||||
}
|
||||
|
||||
for (Throwable e: errors) {
|
||||
throw e;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// This is to compare the original encoding with the current version
|
||||
@Test
|
||||
public void compareOriginal() throws Exception {
|
||||
String generated = generate(TEXT);
|
||||
|
||||
Assert.assertEquals(STRING_ENCODE, generated);
|
||||
|
||||
for (int i = 0; i < generated.length(); i++) {
|
||||
Assert.assertEquals("Chart at " + i + " was " + generated.charAt(i) + " instead of " + STRING_ENCODE.charAt(i), generated.charAt(i), STRING_ENCODE.charAt(i));
|
||||
}
|
||||
}
|
||||
|
||||
/** Use this method to update the encode for the known message */
|
||||
@Ignore
|
||||
@Test
|
||||
public void generate() throws Exception {
|
||||
|
||||
printVariable(TEXT, generate(TEXT));
|
||||
printVariable(SMALLER_TEXT, generate(SMALLER_TEXT));
|
||||
printVariable(BIGGER_TEXT, generate(BIGGER_TEXT));
|
||||
|
||||
}
|
||||
|
||||
private void printVariable(String body, String encode) {
|
||||
System.out.println("// body = \"" + body + "\";");
|
||||
System.out.println("private final String STRING_ENCODE = \"" + encode + "\";");
|
||||
|
||||
}
|
||||
|
||||
public String generate(String body) throws Exception {
|
||||
|
||||
ClientMessageImpl message = new ClientMessageImpl(MESSAGE_TYPE, DURABLE, EXPIRATION, TIMESTAMP, PRIORITY, 10 * 1024);
|
||||
TextMessageUtil.writeBodyText(message.getBodyBuffer(), SimpleString.toSimpleString(body));
|
||||
|
||||
message.setAddress(ADDRESS);
|
||||
message.setUserID(uuid);
|
||||
message.getProperties().putSimpleStringProperty(PROP1_NAME, PROP1_VALUE);
|
||||
|
||||
|
||||
ActiveMQBuffer buffer = ActiveMQBuffers.dynamicBuffer(10 * 1024);
|
||||
message.sendBuffer(buffer.byteBuf(), 0);
|
||||
|
||||
byte[] bytes = new byte[buffer.byteBuf().writerIndex()];
|
||||
buffer.byteBuf().readBytes(bytes);
|
||||
|
||||
return encodeString(bytes);
|
||||
|
||||
// replace the code
|
||||
|
||||
|
||||
}
|
||||
|
||||
private String encodeString(byte[] bytes) {
|
||||
return Base64.encodeBytes(bytes, 0, bytes.length, Base64.DONT_BREAK_LINES | Base64.URL_SAFE);
|
||||
}
|
||||
|
||||
}
|
|
@ -77,20 +77,10 @@
|
|||
<artifactId>artemis-spring-integration</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.activemq</groupId>
|
||||
<artifactId>artemis-vertx-integration</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.activemq.rest</groupId>
|
||||
<artifactId>artemis-rest</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.activemq</groupId>
|
||||
<artifactId>artemis-aerogear-integration</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.activemq</groupId>
|
||||
|
|
|
@ -31,11 +31,13 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
|
||||
import org.apache.activemq.artemis.core.journal.EncoderPersister;
|
||||
import org.apache.activemq.artemis.core.journal.EncodingSupport;
|
||||
import org.apache.activemq.artemis.core.journal.IOCompletion;
|
||||
import org.apache.activemq.artemis.core.journal.Journal;
|
||||
import org.apache.activemq.artemis.core.journal.JournalLoadInformation;
|
||||
import org.apache.activemq.artemis.core.journal.LoaderCallback;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.journal.PreparedTransactionInfo;
|
||||
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
||||
import org.apache.activemq.artemis.core.journal.TransactionFailureCallback;
|
||||
|
@ -366,10 +368,10 @@ public class JDBCJournalImpl extends AbstractJDBCDriver implements Journal {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void appendAddRecord(long id, byte recordType, EncodingSupport record, boolean sync) throws Exception {
|
||||
public void appendAddRecord(long id, byte recordType, Persister persister, Object record, boolean sync) throws Exception {
|
||||
JDBCJournalRecord r = new JDBCJournalRecord(id, JDBCJournalRecord.ADD_RECORD, seq.incrementAndGet());
|
||||
r.setUserRecordType(recordType);
|
||||
r.setRecord(record);
|
||||
r.setRecord(persister, record);
|
||||
r.setSync(sync);
|
||||
appendRecord(r);
|
||||
}
|
||||
|
@ -377,12 +379,13 @@ public class JDBCJournalImpl extends AbstractJDBCDriver implements Journal {
|
|||
@Override
|
||||
public void appendAddRecord(long id,
|
||||
byte recordType,
|
||||
EncodingSupport record,
|
||||
Persister persister,
|
||||
Object record,
|
||||
boolean sync,
|
||||
IOCompletion completionCallback) throws Exception {
|
||||
JDBCJournalRecord r = new JDBCJournalRecord(id, JDBCJournalRecord.ADD_RECORD, seq.incrementAndGet());
|
||||
r.setUserRecordType(recordType);
|
||||
r.setRecord(record);
|
||||
r.setRecord(persister, record);
|
||||
r.setSync(sync);
|
||||
r.setIoCompletion(completionCallback);
|
||||
appendRecord(r);
|
||||
|
@ -398,10 +401,10 @@ public class JDBCJournalImpl extends AbstractJDBCDriver implements Journal {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void appendUpdateRecord(long id, byte recordType, EncodingSupport record, boolean sync) throws Exception {
|
||||
public void appendUpdateRecord(long id, byte recordType, Persister persister, Object record, boolean sync) throws Exception {
|
||||
JDBCJournalRecord r = new JDBCJournalRecord(id, JDBCJournalRecord.UPDATE_RECORD, seq.incrementAndGet());
|
||||
r.setUserRecordType(recordType);
|
||||
r.setRecord(record);
|
||||
r.setRecord(persister, record);
|
||||
r.setSync(sync);
|
||||
appendRecord(r);
|
||||
}
|
||||
|
@ -409,12 +412,13 @@ public class JDBCJournalImpl extends AbstractJDBCDriver implements Journal {
|
|||
@Override
|
||||
public void appendUpdateRecord(long id,
|
||||
byte recordType,
|
||||
EncodingSupport record,
|
||||
Persister persister,
|
||||
Object record,
|
||||
boolean sync,
|
||||
IOCompletion completionCallback) throws Exception {
|
||||
JDBCJournalRecord r = new JDBCJournalRecord(id, JDBCJournalRecord.ADD_RECORD, seq.incrementAndGet());
|
||||
r.setUserRecordType(recordType);
|
||||
r.setRecord(record);
|
||||
r.setRecord(persister, record);
|
||||
r.setSync(sync);
|
||||
r.setIoCompletion(completionCallback);
|
||||
appendRecord(r);
|
||||
|
@ -448,10 +452,11 @@ public class JDBCJournalImpl extends AbstractJDBCDriver implements Journal {
|
|||
public void appendAddRecordTransactional(long txID,
|
||||
long id,
|
||||
byte recordType,
|
||||
EncodingSupport record) throws Exception {
|
||||
Persister persister,
|
||||
Object record) throws Exception {
|
||||
JDBCJournalRecord r = new JDBCJournalRecord(id, JDBCJournalRecord.ADD_RECORD_TX, seq.incrementAndGet());
|
||||
r.setUserRecordType(recordType);
|
||||
r.setRecord(record);
|
||||
r.setRecord(persister, record);
|
||||
r.setTxId(txID);
|
||||
appendRecord(r);
|
||||
}
|
||||
|
@ -469,10 +474,11 @@ public class JDBCJournalImpl extends AbstractJDBCDriver implements Journal {
|
|||
public void appendUpdateRecordTransactional(long txID,
|
||||
long id,
|
||||
byte recordType,
|
||||
EncodingSupport record) throws Exception {
|
||||
Persister persister,
|
||||
Object record) throws Exception {
|
||||
JDBCJournalRecord r = new JDBCJournalRecord(id, JDBCJournalRecord.UPDATE_RECORD_TX, seq.incrementAndGet());
|
||||
r.setUserRecordType(recordType);
|
||||
r.setRecord(record);
|
||||
r.setRecord(persister, record);
|
||||
r.setTxId(txID);
|
||||
appendRecord(r);
|
||||
}
|
||||
|
@ -488,7 +494,7 @@ public class JDBCJournalImpl extends AbstractJDBCDriver implements Journal {
|
|||
@Override
|
||||
public void appendDeleteRecordTransactional(long txID, long id, EncodingSupport record) throws Exception {
|
||||
JDBCJournalRecord r = new JDBCJournalRecord(id, JDBCJournalRecord.DELETE_RECORD_TX, seq.incrementAndGet());
|
||||
r.setRecord(record);
|
||||
r.setRecord(EncoderPersister.getInstance(), record);
|
||||
r.setTxId(txID);
|
||||
appendRecord(r);
|
||||
}
|
||||
|
@ -684,10 +690,6 @@ public class JDBCJournalImpl extends AbstractJDBCDriver implements Journal {
|
|||
return USER_VERSION;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void perfBlast(int pages) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void runDirectJournalBlast() throws Exception {
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
|||
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
|
||||
import org.apache.activemq.artemis.core.journal.EncodingSupport;
|
||||
import org.apache.activemq.artemis.core.journal.IOCompletion;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
||||
import org.apache.activemq.artemis.journal.ActiveMQJournalLogger;
|
||||
import org.apache.activemq.artemis.utils.ActiveMQBufferInputStream;
|
||||
|
@ -237,11 +238,11 @@ class JDBCJournalRecord {
|
|||
this.record = record;
|
||||
}
|
||||
|
||||
public void setRecord(EncodingSupport record) {
|
||||
this.variableSize = record.getEncodeSize();
|
||||
public void setRecord(Persister persister, Object record) {
|
||||
this.variableSize = persister.getEncodeSize(record);
|
||||
|
||||
ActiveMQBuffer encodedBuffer = ActiveMQBuffers.fixedBuffer(variableSize);
|
||||
record.encode(encodedBuffer);
|
||||
persister.encode(encodedBuffer, record);
|
||||
this.record = new ActiveMQBufferInputStream(encodedBuffer);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.activemq.artemis.api.jms.management;
|
|||
import javax.jms.JMSException;
|
||||
import javax.jms.Message;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.client.ClientMessage;
|
||||
import org.apache.activemq.artemis.api.core.management.ManagementHelper;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQMessage;
|
||||
|
||||
|
@ -27,7 +28,7 @@ import org.apache.activemq.artemis.jms.client.ActiveMQMessage;
|
|||
*/
|
||||
public class JMSManagementHelper {
|
||||
|
||||
private static org.apache.activemq.artemis.api.core.Message getCoreMessage(final Message jmsMessage) {
|
||||
private static ClientMessage getCoreMessage(final Message jmsMessage) {
|
||||
if (jmsMessage instanceof ActiveMQMessage == false) {
|
||||
throw new IllegalArgumentException("Cannot send a foreign message as a management message " + jmsMessage.getClass().getName());
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQException;
|
|||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.client.ClientMessage;
|
||||
import org.apache.activemq.artemis.api.core.client.ClientSession;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageImpl;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
|
||||
import static org.apache.activemq.artemis.reader.BytesMessageUtil.bytesMessageReset;
|
||||
import static org.apache.activemq.artemis.reader.BytesMessageUtil.bytesReadBoolean;
|
||||
|
@ -374,7 +374,7 @@ public class ActiveMQBytesMessage extends ActiveMQMessage implements BytesMessag
|
|||
if (bodyLength == 0)
|
||||
return null;
|
||||
byte[] dst = new byte[bodyLength];
|
||||
message.getBodyBuffer().getBytes(MessageImpl.BODY_OFFSET, dst);
|
||||
message.getBodyBuffer().getBytes(CoreMessage.BODY_OFFSET, dst);
|
||||
return (T) dst;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,7 +43,7 @@ import org.apache.activemq.artemis.api.core.SimpleString;
|
|||
import org.apache.activemq.artemis.api.core.client.ClientMessage;
|
||||
import org.apache.activemq.artemis.api.core.client.ClientSession;
|
||||
import org.apache.activemq.artemis.api.jms.ActiveMQJMSConstants;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientMessageInternal;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.reader.MessageUtil;
|
||||
import org.apache.activemq.artemis.utils.UUID;
|
||||
|
@ -293,7 +293,7 @@ public class ActiveMQMessage implements javax.jms.Message {
|
|||
@Override
|
||||
public String getJMSMessageID() {
|
||||
if (msgID == null) {
|
||||
UUID uid = message.getUserID();
|
||||
UUID uid = (UUID)message.getUserID();
|
||||
|
||||
msgID = uid == null ? null : "ID:" + uid.toString();
|
||||
}
|
||||
|
@ -397,7 +397,7 @@ public class ActiveMQMessage implements javax.jms.Message {
|
|||
@Override
|
||||
public Destination getJMSDestination() throws JMSException {
|
||||
if (dest == null) {
|
||||
SimpleString address = message.getAddress();
|
||||
SimpleString address = message.getAddressSimpleString();
|
||||
String prefix = "";
|
||||
if (message.containsProperty(org.apache.activemq.artemis.api.core.Message.HDR_ROUTING_TYPE)) {
|
||||
RoutingType routingType = RoutingType.getType(message.getByteProperty(org.apache.activemq.artemis.api.core.Message.HDR_ROUTING_TYPE));
|
||||
|
@ -756,7 +756,7 @@ public class ActiveMQMessage implements javax.jms.Message {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected <T> T getBodyInternal(Class<T> c) throws MessageFormatException {
|
||||
InputStream is = ((MessageInternal) message).getBodyInputStream();
|
||||
InputStream is = ((ClientMessageInternal) message).getBodyInputStream();
|
||||
try {
|
||||
ObjectInputStream ois = new ObjectInputStream(is);
|
||||
return (T) ois.readObject();
|
||||
|
|
|
@ -19,7 +19,8 @@ package org.apache.activemq.artemis.jms.transaction;
|
|||
import javax.transaction.xa.Xid;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.core.transaction.TransactionDetail;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQBytesMessage;
|
||||
|
@ -36,8 +37,11 @@ public class JMSTransactionDetail extends TransactionDetail {
|
|||
}
|
||||
|
||||
@Override
|
||||
public String decodeMessageType(ServerMessage msg) {
|
||||
int type = msg.getType();
|
||||
public String decodeMessageType(Message msg) {
|
||||
if (!(msg instanceof ICoreMessage)) {
|
||||
return "N/A";
|
||||
}
|
||||
int type = ((ICoreMessage) msg).getType();
|
||||
switch (type) {
|
||||
case ActiveMQMessage.TYPE: // 0
|
||||
return "Default";
|
||||
|
@ -57,7 +61,7 @@ public class JMSTransactionDetail extends TransactionDetail {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> decodeMessageProperties(ServerMessage msg) {
|
||||
public Map<String, Object> decodeMessageProperties(Message msg) {
|
||||
try {
|
||||
return ActiveMQMessage.coreMaptoJMSMap(msg.toMap());
|
||||
} catch (Throwable t) {
|
||||
|
|
|
@ -0,0 +1,51 @@
|
|||
/**
|
||||
* 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.journal;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
|
||||
/** This is a facade between the new Persister and the former EncodingSupport.
|
||||
* Methods using the old interface will use this as a facade to provide the previous semantic. */
|
||||
public class EncoderPersister implements Persister<EncodingSupport> {
|
||||
|
||||
private static final EncoderPersister theInstance = new EncoderPersister();
|
||||
|
||||
private EncoderPersister() {
|
||||
}
|
||||
|
||||
public static EncoderPersister getInstance() {
|
||||
return theInstance;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getEncodeSize(EncodingSupport record) {
|
||||
return record.getEncodeSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void encode(ActiveMQBuffer buffer, EncodingSupport record) {
|
||||
record.encode(buffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public EncodingSupport decode(ActiveMQBuffer buffer, EncodingSupport record) {
|
||||
record.decode(buffer);
|
||||
return record;
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ import java.util.Map;
|
|||
|
||||
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
|
||||
import org.apache.activemq.artemis.core.journal.impl.JournalFile;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQComponent;
|
||||
|
||||
/**
|
||||
|
@ -60,23 +61,49 @@ public interface Journal extends ActiveMQComponent {
|
|||
|
||||
void appendAddRecord(long id, byte recordType, byte[] record, boolean sync) throws Exception;
|
||||
|
||||
void appendAddRecord(long id, byte recordType, EncodingSupport record, boolean sync) throws Exception;
|
||||
default void appendAddRecord(long id, byte recordType, EncodingSupport record, boolean sync) throws Exception {
|
||||
appendAddRecord(id, recordType, EncoderPersister.getInstance(), record, sync);
|
||||
}
|
||||
|
||||
void appendAddRecord(long id, byte recordType, Persister persister, Object record, boolean sync) throws Exception;
|
||||
|
||||
void appendAddRecord(long id,
|
||||
byte recordType,
|
||||
EncodingSupport record,
|
||||
Persister persister,
|
||||
Object record,
|
||||
boolean sync,
|
||||
IOCompletion completionCallback) throws Exception;
|
||||
|
||||
default void appendAddRecord(long id,
|
||||
byte recordType,
|
||||
EncodingSupport record,
|
||||
boolean sync,
|
||||
IOCompletion completionCallback) throws Exception {
|
||||
appendAddRecord(id, recordType, EncoderPersister.getInstance(), record, sync, completionCallback);
|
||||
}
|
||||
|
||||
void appendUpdateRecord(long id, byte recordType, byte[] record, boolean sync) throws Exception;
|
||||
|
||||
void appendUpdateRecord(long id, byte recordType, EncodingSupport record, boolean sync) throws Exception;
|
||||
default void appendUpdateRecord(long id, byte recordType, EncodingSupport record, boolean sync) throws Exception {
|
||||
appendUpdateRecord(id, recordType, EncoderPersister.getInstance(), record, sync);
|
||||
}
|
||||
|
||||
void appendUpdateRecord(long id,
|
||||
void appendUpdateRecord(long id, byte recordType, Persister persister, Object record, boolean sync) throws Exception;
|
||||
|
||||
default void appendUpdateRecord(long id,
|
||||
byte recordType,
|
||||
EncodingSupport record,
|
||||
boolean sync,
|
||||
IOCompletion completionCallback) throws Exception;
|
||||
IOCompletion completionCallback) throws Exception {
|
||||
appendUpdateRecord(id, recordType, EncoderPersister.getInstance(), record, sync, completionCallback);
|
||||
}
|
||||
|
||||
void appendUpdateRecord(final long id,
|
||||
final byte recordType,
|
||||
final Persister persister,
|
||||
final Object record,
|
||||
final boolean sync,
|
||||
final IOCompletion callback) throws Exception;
|
||||
|
||||
void appendDeleteRecord(long id, boolean sync) throws Exception;
|
||||
|
||||
|
@ -86,11 +113,23 @@ public interface Journal extends ActiveMQComponent {
|
|||
|
||||
void appendAddRecordTransactional(long txID, long id, byte recordType, byte[] record) throws Exception;
|
||||
|
||||
void appendAddRecordTransactional(long txID, long id, byte recordType, EncodingSupport record) throws Exception;
|
||||
default void appendAddRecordTransactional(long txID, long id, byte recordType, EncodingSupport record) throws Exception {
|
||||
appendAddRecordTransactional(txID, id, recordType, EncoderPersister.getInstance(), record);
|
||||
}
|
||||
|
||||
void appendAddRecordTransactional(final long txID,
|
||||
final long id,
|
||||
final byte recordType,
|
||||
final Persister persister,
|
||||
final Object record) throws Exception;
|
||||
|
||||
void appendUpdateRecordTransactional(long txID, long id, byte recordType, byte[] record) throws Exception;
|
||||
|
||||
void appendUpdateRecordTransactional(long txID, long id, byte recordType, EncodingSupport record) throws Exception;
|
||||
default void appendUpdateRecordTransactional(long txID, long id, byte recordType, EncodingSupport record) throws Exception {
|
||||
appendUpdateRecordTransactional(txID, id, recordType, EncoderPersister.getInstance(), record);
|
||||
}
|
||||
|
||||
void appendUpdateRecordTransactional(long txID, long id, byte recordType, Persister persister, Object record) throws Exception;
|
||||
|
||||
void appendDeleteRecordTransactional(long txID, long id, byte[] record) throws Exception;
|
||||
|
||||
|
@ -165,8 +204,6 @@ public interface Journal extends ActiveMQComponent {
|
|||
|
||||
int getUserVersion();
|
||||
|
||||
void perfBlast(int pages);
|
||||
|
||||
void runDirectJournalBlast() throws Exception;
|
||||
|
||||
/**
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
|
|||
import org.apache.activemq.artemis.api.core.Pair;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFile;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
|
||||
import org.apache.activemq.artemis.core.journal.EncoderPersister;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.ByteArrayEncoding;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.JournalAddRecord;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.JournalInternalRecord;
|
||||
|
@ -127,7 +128,7 @@ public abstract class AbstractJournalUpdateTask implements JournalReaderCallback
|
|||
}
|
||||
}
|
||||
|
||||
JournalInternalRecord controlRecord = new JournalAddRecord(true, 1, (byte) 0, new ByteArrayEncoding(filesToRename.toByteBuffer().array()));
|
||||
JournalInternalRecord controlRecord = new JournalAddRecord(true, 1, (byte) 0, EncoderPersister.getInstance(), new ByteArrayEncoding(filesToRename.toByteBuffer().array()));
|
||||
|
||||
ActiveMQBuffer renameBuffer = ActiveMQBuffers.dynamicBuffer(filesToRename.writerIndex());
|
||||
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.activemq.artemis.core.journal.IOCompletion;
|
|||
import org.apache.activemq.artemis.core.journal.Journal;
|
||||
import org.apache.activemq.artemis.core.journal.JournalLoadInformation;
|
||||
import org.apache.activemq.artemis.core.journal.LoaderCallback;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.journal.PreparedTransactionInfo;
|
||||
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
||||
import org.apache.activemq.artemis.core.journal.TransactionFailureCallback;
|
||||
|
@ -90,10 +91,11 @@ public final class FileWrapperJournal extends JournalBase {
|
|||
@Override
|
||||
public void appendAddRecord(long id,
|
||||
byte recordType,
|
||||
EncodingSupport record,
|
||||
Persister persister,
|
||||
Object record,
|
||||
boolean sync,
|
||||
IOCompletion callback) throws Exception {
|
||||
JournalInternalRecord addRecord = new JournalAddRecord(true, id, recordType, record);
|
||||
JournalInternalRecord addRecord = new JournalAddRecord(true, id, recordType, persister, record);
|
||||
|
||||
writeRecord(addRecord, sync, callback);
|
||||
}
|
||||
|
@ -144,19 +146,21 @@ public final class FileWrapperJournal extends JournalBase {
|
|||
public void appendAddRecordTransactional(long txID,
|
||||
long id,
|
||||
byte recordType,
|
||||
EncodingSupport record) throws Exception {
|
||||
Persister persister,
|
||||
Object record) throws Exception {
|
||||
count(txID);
|
||||
JournalInternalRecord addRecord = new JournalAddRecordTX(true, txID, id, recordType, record);
|
||||
JournalInternalRecord addRecord = new JournalAddRecordTX(true, txID, id, recordType, persister, record);
|
||||
writeRecord(addRecord, false, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void appendUpdateRecord(long id,
|
||||
byte recordType,
|
||||
EncodingSupport record,
|
||||
Persister persister,
|
||||
Object record,
|
||||
boolean sync,
|
||||
IOCompletion callback) throws Exception {
|
||||
JournalInternalRecord updateRecord = new JournalAddRecord(false, id, recordType, record);
|
||||
JournalInternalRecord updateRecord = new JournalAddRecord(false, id, recordType, persister, record);
|
||||
writeRecord(updateRecord, sync, callback);
|
||||
}
|
||||
|
||||
|
@ -164,9 +168,10 @@ public final class FileWrapperJournal extends JournalBase {
|
|||
public void appendUpdateRecordTransactional(long txID,
|
||||
long id,
|
||||
byte recordType,
|
||||
EncodingSupport record) throws Exception {
|
||||
Persister persister,
|
||||
Object record) throws Exception {
|
||||
count(txID);
|
||||
JournalInternalRecord updateRecordTX = new JournalAddRecordTX(false, txID, id, recordType, record);
|
||||
JournalInternalRecord updateRecordTX = new JournalAddRecordTX(false, txID, id, recordType, persister, record);
|
||||
writeRecord(updateRecordTX, false, null);
|
||||
}
|
||||
|
||||
|
@ -260,11 +265,6 @@ public final class FileWrapperJournal extends JournalBase {
|
|||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void perfBlast(int pages) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void runDirectJournalBlast() throws Exception {
|
||||
throw new UnsupportedOperationException();
|
||||
|
|
|
@ -21,6 +21,7 @@ import org.apache.activemq.artemis.core.io.DummyCallback;
|
|||
import org.apache.activemq.artemis.core.journal.EncodingSupport;
|
||||
import org.apache.activemq.artemis.core.journal.IOCompletion;
|
||||
import org.apache.activemq.artemis.core.journal.Journal;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.ByteArrayEncoding;
|
||||
|
||||
abstract class JournalBase implements Journal {
|
||||
|
@ -36,69 +37,16 @@ abstract class JournalBase implements Journal {
|
|||
this.fileSize = fileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public abstract void appendAddRecord(final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record,
|
||||
final boolean sync,
|
||||
final IOCompletion callback) throws Exception;
|
||||
|
||||
@Override
|
||||
public abstract void appendAddRecordTransactional(final long txID,
|
||||
final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record) throws Exception;
|
||||
|
||||
@Override
|
||||
public abstract void appendCommitRecord(final long txID,
|
||||
final boolean sync,
|
||||
final IOCompletion callback,
|
||||
boolean lineUpContext) throws Exception;
|
||||
|
||||
@Override
|
||||
public abstract void appendDeleteRecord(final long id,
|
||||
final boolean sync,
|
||||
final IOCompletion callback) throws Exception;
|
||||
|
||||
@Override
|
||||
public abstract void appendDeleteRecordTransactional(final long txID,
|
||||
final long id,
|
||||
final EncodingSupport record) throws Exception;
|
||||
|
||||
@Override
|
||||
public abstract void appendPrepareRecord(final long txID,
|
||||
final EncodingSupport transactionData,
|
||||
final boolean sync,
|
||||
final IOCompletion callback) throws Exception;
|
||||
|
||||
@Override
|
||||
public abstract void appendUpdateRecord(final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record,
|
||||
final boolean sync,
|
||||
final IOCompletion callback) throws Exception;
|
||||
|
||||
@Override
|
||||
public abstract void appendUpdateRecordTransactional(final long txID,
|
||||
final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record) throws Exception;
|
||||
|
||||
@Override
|
||||
public abstract void appendRollbackRecord(final long txID,
|
||||
final boolean sync,
|
||||
final IOCompletion callback) throws Exception;
|
||||
|
||||
@Override
|
||||
public void appendAddRecord(long id, byte recordType, byte[] record, boolean sync) throws Exception {
|
||||
appendAddRecord(id, recordType, new ByteArrayEncoding(record), sync);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void appendAddRecord(long id, byte recordType, EncodingSupport record, boolean sync) throws Exception {
|
||||
public void appendAddRecord(long id, byte recordType, Persister persister, Object record, boolean sync) throws Exception {
|
||||
SyncIOCompletion callback = getSyncCallback(sync);
|
||||
|
||||
appendAddRecord(id, recordType, record, sync, callback);
|
||||
appendAddRecord(id, recordType, persister, record, sync, callback);
|
||||
|
||||
if (callback != null) {
|
||||
callback.waitCompletion();
|
||||
|
@ -176,11 +124,12 @@ abstract class JournalBase implements Journal {
|
|||
@Override
|
||||
public void appendUpdateRecord(final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record,
|
||||
final Persister persister,
|
||||
final Object record,
|
||||
final boolean sync) throws Exception {
|
||||
SyncIOCompletion callback = getSyncCallback(sync);
|
||||
|
||||
appendUpdateRecord(id, recordType, record, sync, callback);
|
||||
appendUpdateRecord(id, recordType, persister, record, sync, callback);
|
||||
|
||||
if (callback != null) {
|
||||
callback.waitCompletion();
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
|
|||
import org.apache.activemq.artemis.api.core.Pair;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFile;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFileFactory;
|
||||
import org.apache.activemq.artemis.core.journal.EncoderPersister;
|
||||
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.ByteArrayEncoding;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.JournalAddRecord;
|
||||
|
@ -252,7 +253,7 @@ public class JournalCompactor extends AbstractJournalUpdateTask implements Journ
|
|||
@Override
|
||||
public void onReadAddRecord(final RecordInfo info) throws Exception {
|
||||
if (lookupRecord(info.id)) {
|
||||
JournalInternalRecord addRecord = new JournalAddRecord(true, info.id, info.getUserRecordType(), new ByteArrayEncoding(info.data));
|
||||
JournalInternalRecord addRecord = new JournalAddRecord(true, info.id, info.getUserRecordType(), EncoderPersister.getInstance(), new ByteArrayEncoding(info.data));
|
||||
addRecord.setCompactCount((short) (info.compactCount + 1));
|
||||
|
||||
checkSize(addRecord.getEncodeSize(), info.compactCount);
|
||||
|
@ -268,7 +269,7 @@ public class JournalCompactor extends AbstractJournalUpdateTask implements Journ
|
|||
if (pendingTransactions.get(transactionID) != null || lookupRecord(info.id)) {
|
||||
JournalTransaction newTransaction = getNewJournalTransaction(transactionID);
|
||||
|
||||
JournalInternalRecord record = new JournalAddRecordTX(true, transactionID, info.id, info.getUserRecordType(), new ByteArrayEncoding(info.data));
|
||||
JournalInternalRecord record = new JournalAddRecordTX(true, transactionID, info.id, info.getUserRecordType(), EncoderPersister.getInstance(),new ByteArrayEncoding(info.data));
|
||||
|
||||
record.setCompactCount((short) (info.compactCount + 1));
|
||||
|
||||
|
@ -374,7 +375,7 @@ public class JournalCompactor extends AbstractJournalUpdateTask implements Journ
|
|||
@Override
|
||||
public void onReadUpdateRecord(final RecordInfo info) throws Exception {
|
||||
if (lookupRecord(info.id)) {
|
||||
JournalInternalRecord updateRecord = new JournalAddRecord(false, info.id, info.userRecordType, new ByteArrayEncoding(info.data));
|
||||
JournalInternalRecord updateRecord = new JournalAddRecord(false, info.id, info.userRecordType, EncoderPersister.getInstance(), new ByteArrayEncoding(info.data));
|
||||
|
||||
updateRecord.setCompactCount((short) (info.compactCount + 1));
|
||||
|
||||
|
@ -397,7 +398,7 @@ public class JournalCompactor extends AbstractJournalUpdateTask implements Journ
|
|||
if (pendingTransactions.get(transactionID) != null || lookupRecord(info.id)) {
|
||||
JournalTransaction newTransaction = getNewJournalTransaction(transactionID);
|
||||
|
||||
JournalInternalRecord updateRecordTX = new JournalAddRecordTX(false, transactionID, info.id, info.userRecordType, new ByteArrayEncoding(info.data));
|
||||
JournalInternalRecord updateRecordTX = new JournalAddRecordTX(false, transactionID, info.id, info.userRecordType, EncoderPersister.getInstance(), new ByteArrayEncoding(info.data));
|
||||
|
||||
updateRecordTX.setCompactCount((short) (info.compactCount + 1));
|
||||
|
||||
|
|
|
@ -57,11 +57,11 @@ import org.apache.activemq.artemis.core.journal.EncodingSupport;
|
|||
import org.apache.activemq.artemis.core.journal.IOCompletion;
|
||||
import org.apache.activemq.artemis.core.journal.JournalLoadInformation;
|
||||
import org.apache.activemq.artemis.core.journal.LoaderCallback;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.journal.PreparedTransactionInfo;
|
||||
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
||||
import org.apache.activemq.artemis.core.journal.TestableJournal;
|
||||
import org.apache.activemq.artemis.core.journal.TransactionFailureCallback;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.ByteArrayEncoding;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.JournalAddRecord;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.JournalAddRecordTX;
|
||||
import org.apache.activemq.artemis.core.journal.impl.dataformat.JournalCompleteRecordTX;
|
||||
|
@ -713,7 +713,8 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
@Override
|
||||
public void appendAddRecord(final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record,
|
||||
final Persister persister,
|
||||
final Object record,
|
||||
final boolean sync,
|
||||
final IOCompletion callback) throws Exception {
|
||||
checkJournalIsLoaded();
|
||||
|
@ -727,7 +728,7 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
public void run() {
|
||||
journalLock.readLock().lock();
|
||||
try {
|
||||
JournalInternalRecord addRecord = new JournalAddRecord(true, id, recordType, record);
|
||||
JournalInternalRecord addRecord = new JournalAddRecord(true, id, recordType, persister, record);
|
||||
JournalFile usedFile = appendRecord(addRecord, false, sync, null, callback);
|
||||
records.put(id, new JournalRecord(usedFile, addRecord.getEncodeSize()));
|
||||
|
||||
|
@ -762,7 +763,8 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
@Override
|
||||
public void appendUpdateRecord(final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record,
|
||||
final Persister persister,
|
||||
final Object record,
|
||||
final boolean sync,
|
||||
final IOCompletion callback) throws Exception {
|
||||
checkJournalIsLoaded();
|
||||
|
@ -777,7 +779,7 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
journalLock.readLock().lock();
|
||||
try {
|
||||
JournalRecord jrnRecord = records.get(id);
|
||||
JournalInternalRecord updateRecord = new JournalAddRecord(false, id, recordType, record);
|
||||
JournalInternalRecord updateRecord = new JournalAddRecord(false, id, recordType, persister, record);
|
||||
JournalFile usedFile = appendRecord(updateRecord, false, sync, null, callback);
|
||||
|
||||
if (logger.isTraceEnabled()) {
|
||||
|
@ -873,7 +875,8 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
public void appendAddRecordTransactional(final long txID,
|
||||
final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record) throws Exception {
|
||||
final Persister persister,
|
||||
final Object record) throws Exception {
|
||||
checkJournalIsLoaded();
|
||||
|
||||
final JournalTransaction tx = getTransactionInfo(txID);
|
||||
|
@ -885,7 +888,7 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
public void run() {
|
||||
journalLock.readLock().lock();
|
||||
try {
|
||||
JournalInternalRecord addRecord = new JournalAddRecordTX(true, txID, id, recordType, record);
|
||||
JournalInternalRecord addRecord = new JournalAddRecordTX(true, txID, id, recordType, persister, record);
|
||||
JournalFile usedFile = appendRecord(addRecord, false, false, tx, null);
|
||||
|
||||
if (logger.isTraceEnabled()) {
|
||||
|
@ -952,7 +955,8 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
public void appendUpdateRecordTransactional(final long txID,
|
||||
final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record) throws Exception {
|
||||
final Persister persister,
|
||||
final Object record) throws Exception {
|
||||
checkJournalIsLoaded();
|
||||
|
||||
final JournalTransaction tx = getTransactionInfo(txID);
|
||||
|
@ -965,7 +969,7 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
journalLock.readLock().lock();
|
||||
try {
|
||||
|
||||
JournalInternalRecord updateRecordTX = new JournalAddRecordTX( false, txID, id, recordType, record );
|
||||
JournalInternalRecord updateRecordTX = new JournalAddRecordTX( false, txID, id, recordType, persister, record );
|
||||
JournalFile usedFile = appendRecord( updateRecordTX, false, false, tx, null );
|
||||
|
||||
if ( logger.isTraceEnabled() ) {
|
||||
|
@ -2165,45 +2169,6 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void perfBlast(final int pages) {
|
||||
|
||||
checkJournalIsLoaded();
|
||||
|
||||
final ByteArrayEncoding byteEncoder = new ByteArrayEncoding(new byte[128 * 1024]);
|
||||
|
||||
final JournalInternalRecord blastRecord = new JournalInternalRecord() {
|
||||
|
||||
@Override
|
||||
public int getEncodeSize() {
|
||||
return byteEncoder.getEncodeSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void encode(final ActiveMQBuffer buffer) {
|
||||
byteEncoder.encode(buffer);
|
||||
}
|
||||
};
|
||||
|
||||
appendExecutor.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
journalLock.readLock().lock();
|
||||
try {
|
||||
|
||||
for (int i = 0; i < pages; i++) {
|
||||
appendRecord(blastRecord, false, false, null, null);
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
ActiveMQJournalLogger.LOGGER.failedToPerfBlast(e);
|
||||
} finally {
|
||||
journalLock.readLock().unlock();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
// ActiveMQComponent implementation
|
||||
// ---------------------------------------------------
|
||||
|
||||
|
@ -2921,5 +2886,4 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
|
|||
public int getCompactCount() {
|
||||
return compactCount;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,14 +17,16 @@
|
|||
package org.apache.activemq.artemis.core.journal.impl.dataformat;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.journal.EncodingSupport;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.journal.impl.JournalImpl;
|
||||
|
||||
public class JournalAddRecord extends JournalInternalRecord {
|
||||
|
||||
protected final long id;
|
||||
|
||||
protected final EncodingSupport record;
|
||||
protected final Persister persister;
|
||||
|
||||
protected final Object record;
|
||||
|
||||
protected final byte recordType;
|
||||
|
||||
|
@ -35,7 +37,7 @@ public class JournalAddRecord extends JournalInternalRecord {
|
|||
* @param recordType
|
||||
* @param record
|
||||
*/
|
||||
public JournalAddRecord(final boolean add, final long id, final byte recordType, final EncodingSupport record) {
|
||||
public JournalAddRecord(final boolean add, final long id, final byte recordType, final Persister persister, Object record) {
|
||||
this.id = id;
|
||||
|
||||
this.record = record;
|
||||
|
@ -43,6 +45,8 @@ public class JournalAddRecord extends JournalInternalRecord {
|
|||
this.recordType = recordType;
|
||||
|
||||
this.add = add;
|
||||
|
||||
this.persister = persister;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -59,17 +63,19 @@ public class JournalAddRecord extends JournalInternalRecord {
|
|||
|
||||
buffer.writeLong(id);
|
||||
|
||||
buffer.writeInt(record.getEncodeSize());
|
||||
int recordEncodeSize = persister.getEncodeSize(record);
|
||||
|
||||
buffer.writeInt(persister.getEncodeSize(record));
|
||||
|
||||
buffer.writeByte(recordType);
|
||||
|
||||
record.encode(buffer);
|
||||
persister.encode(buffer, record);
|
||||
|
||||
buffer.writeInt(getEncodeSize());
|
||||
buffer.writeInt(recordEncodeSize + JournalImpl.SIZE_ADD_RECORD + 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getEncodeSize() {
|
||||
return JournalImpl.SIZE_ADD_RECORD + record.getEncodeSize() + 1;
|
||||
return JournalImpl.SIZE_ADD_RECORD + persister.getEncodeSize(record) + 1;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
package org.apache.activemq.artemis.core.journal.impl.dataformat;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.journal.EncodingSupport;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.journal.impl.JournalImpl;
|
||||
|
||||
public class JournalAddRecordTX extends JournalInternalRecord {
|
||||
|
@ -26,7 +26,9 @@ public class JournalAddRecordTX extends JournalInternalRecord {
|
|||
|
||||
private final long id;
|
||||
|
||||
private final EncodingSupport record;
|
||||
protected final Persister persister;
|
||||
|
||||
protected final Object record;
|
||||
|
||||
private final byte recordType;
|
||||
|
||||
|
@ -41,12 +43,15 @@ public class JournalAddRecordTX extends JournalInternalRecord {
|
|||
final long txID,
|
||||
final long id,
|
||||
final byte recordType,
|
||||
final EncodingSupport record) {
|
||||
final Persister persister,
|
||||
Object record) {
|
||||
|
||||
this.txID = txID;
|
||||
|
||||
this.id = id;
|
||||
|
||||
this.persister = persister;
|
||||
|
||||
this.record = record;
|
||||
|
||||
this.recordType = recordType;
|
||||
|
@ -70,17 +75,17 @@ public class JournalAddRecordTX extends JournalInternalRecord {
|
|||
|
||||
buffer.writeLong(id);
|
||||
|
||||
buffer.writeInt(record.getEncodeSize());
|
||||
buffer.writeInt(persister.getEncodeSize(record));
|
||||
|
||||
buffer.writeByte(recordType);
|
||||
|
||||
record.encode(buffer);
|
||||
persister.encode(buffer, record);
|
||||
|
||||
buffer.writeInt(getEncodeSize());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getEncodeSize() {
|
||||
return JournalImpl.SIZE_ADD_RECORD_TX + record.getEncodeSize() + 1;
|
||||
return JournalImpl.SIZE_ADD_RECORD_TX + persister.getEncodeSize(record) + 1;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,872 @@
|
|||
/*
|
||||
* 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.protocol.amqp.broker;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQPropertyConversionException;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.RefCountMessage;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPConverter;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.TLSEncode;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
import org.apache.qpid.proton.amqp.Symbol;
|
||||
import org.apache.qpid.proton.amqp.UnsignedInteger;
|
||||
import org.apache.qpid.proton.amqp.messaging.ApplicationProperties;
|
||||
import org.apache.qpid.proton.amqp.messaging.DeliveryAnnotations;
|
||||
import org.apache.qpid.proton.amqp.messaging.Header;
|
||||
import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
|
||||
import org.apache.qpid.proton.amqp.messaging.Properties;
|
||||
import org.apache.qpid.proton.amqp.messaging.Section;
|
||||
import org.apache.qpid.proton.codec.DecoderImpl;
|
||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||
import org.apache.qpid.proton.message.Message;
|
||||
import org.apache.qpid.proton.message.impl.MessageImpl;
|
||||
|
||||
// see https://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-messaging-v1.0-os.html#section-message-format
|
||||
public class AMQPMessage extends RefCountMessage {
|
||||
|
||||
final long messageFormat;
|
||||
ByteBuf data;
|
||||
boolean bufferValid;
|
||||
byte type;
|
||||
long messageID;
|
||||
String address;
|
||||
MessageImpl protonMessage;
|
||||
private volatile int memoryEstimate = -1;
|
||||
private long expiration = 0;
|
||||
// this is to store where to start sending bytes, ignoring header and delivery annotations.
|
||||
private int sendFrom = -1;
|
||||
private boolean parsedHeaders = false;
|
||||
private Header _header;
|
||||
private DeliveryAnnotations _deliveryAnnotations;
|
||||
private MessageAnnotations _messageAnnotations;
|
||||
private Properties _properties;
|
||||
private ApplicationProperties applicationProperties;
|
||||
private long scheduledTime = -1;
|
||||
|
||||
public AMQPMessage(long messageFormat, byte[] data) {
|
||||
this.data = Unpooled.wrappedBuffer(data);
|
||||
this.messageFormat = messageFormat;
|
||||
this.bufferValid = true;
|
||||
|
||||
}
|
||||
|
||||
/** for persistence reload */
|
||||
public AMQPMessage(long messageFormat) {
|
||||
this.messageFormat = messageFormat;
|
||||
this.bufferValid = false;
|
||||
|
||||
}
|
||||
|
||||
public AMQPMessage(long messageFormat, Message message) {
|
||||
this.messageFormat = messageFormat;
|
||||
this.protonMessage = (MessageImpl)message;
|
||||
|
||||
}
|
||||
|
||||
public AMQPMessage(Message message) {
|
||||
this(0, message);
|
||||
}
|
||||
|
||||
public MessageImpl getProtonMessage() {
|
||||
if (protonMessage == null) {
|
||||
protonMessage = (MessageImpl) Message.Factory.create();
|
||||
|
||||
if (data != null) {
|
||||
data.readerIndex(0);
|
||||
protonMessage.decode(data.nioBuffer());
|
||||
this._header = protonMessage.getHeader();
|
||||
protonMessage.setHeader(null);
|
||||
}
|
||||
}
|
||||
|
||||
return protonMessage;
|
||||
}
|
||||
|
||||
private void initalizeObjects() {
|
||||
if (protonMessage == null) {
|
||||
if (data == null) {
|
||||
this.sendFrom = -1;
|
||||
_header = new Header();
|
||||
_deliveryAnnotations = new DeliveryAnnotations(new HashMap<>());
|
||||
_properties = new Properties();
|
||||
this.applicationProperties = new ApplicationProperties(new HashMap<>());
|
||||
this.protonMessage = (MessageImpl)Message.Factory.create();
|
||||
this.protonMessage.setApplicationProperties(applicationProperties);
|
||||
this.protonMessage.setDeliveryAnnotations(_deliveryAnnotations);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Map getApplicationPropertiesMap() {
|
||||
|
||||
ApplicationProperties appMap = getApplicationProperties();
|
||||
Map map = null;
|
||||
|
||||
if (appMap != null) {
|
||||
map = appMap.getValue();
|
||||
}
|
||||
|
||||
if (map == null) {
|
||||
return Collections.emptyMap();
|
||||
} else {
|
||||
return map;
|
||||
}
|
||||
}
|
||||
|
||||
private ApplicationProperties getApplicationProperties() {
|
||||
parseHeaders();
|
||||
return applicationProperties;
|
||||
}
|
||||
|
||||
private void parseHeaders() {
|
||||
if (!parsedHeaders) {
|
||||
if (data == null) {
|
||||
initalizeObjects();
|
||||
} else {
|
||||
partialDecode(data.nioBuffer());
|
||||
}
|
||||
parsedHeaders = true;
|
||||
}
|
||||
}
|
||||
|
||||
public MessageAnnotations getMessageAnnotations() {
|
||||
parseHeaders();
|
||||
return _messageAnnotations;
|
||||
}
|
||||
|
||||
public Header getHeader() {
|
||||
parseHeaders();
|
||||
return _header;
|
||||
}
|
||||
|
||||
public Properties getProperties() {
|
||||
parseHeaders();
|
||||
return _properties;
|
||||
}
|
||||
|
||||
private Object getSymbol(String symbol) {
|
||||
return getSymbol(Symbol.getSymbol(symbol));
|
||||
}
|
||||
|
||||
private Object getSymbol(Symbol symbol) {
|
||||
MessageAnnotations annotations = getMessageAnnotations();
|
||||
Map mapAnnotations = annotations != null ? annotations.getValue() : null;
|
||||
if (mapAnnotations != null) {
|
||||
return mapAnnotations.get(symbol);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
private void setSymbol(String symbol, Object value) {
|
||||
setSymbol(Symbol.getSymbol(symbol), value);
|
||||
}
|
||||
|
||||
private void setSymbol(Symbol symbol, Object value) {
|
||||
MessageAnnotations annotations = getMessageAnnotations();
|
||||
Map mapAnnotations = annotations != null ? annotations.getValue() : null;
|
||||
if (mapAnnotations != null) {
|
||||
mapAnnotations.put(symbol, value);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RoutingType getRouteType() {
|
||||
|
||||
/* TODO-now How to use this properly
|
||||
switch (((Byte)type).byteValue()) {
|
||||
case AMQPMessageSupport.QUEUE_TYPE:
|
||||
case AMQPMessageSupport.TEMP_QUEUE_TYPE:
|
||||
return RoutingType.ANYCAST;
|
||||
|
||||
case AMQPMessageSupport.TOPIC_TYPE:
|
||||
case AMQPMessageSupport.TEMP_TOPIC_TYPE:
|
||||
return RoutingType.MULTICAST;
|
||||
default:
|
||||
return null;
|
||||
} */
|
||||
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public Long getScheduledDeliveryTime() {
|
||||
|
||||
if (scheduledTime < 0) {
|
||||
Object objscheduledTime = getSymbol("x-opt-delivery-time");
|
||||
Object objdelay = getSymbol("x-opt-delivery-delay");
|
||||
|
||||
if (objscheduledTime != null && objscheduledTime instanceof Number) {
|
||||
this.scheduledTime = ((Number) objscheduledTime).longValue();
|
||||
} else if (objdelay != null && objdelay instanceof Number) {
|
||||
this.scheduledTime = System.currentTimeMillis() + ((Number) objdelay).longValue();
|
||||
} else {
|
||||
this.scheduledTime = 0;
|
||||
}
|
||||
}
|
||||
|
||||
return scheduledTime == 0 ? null : scheduledTime;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AMQPMessage setScheduledDeliveryTime(Long time) {
|
||||
parseHeaders();
|
||||
setSymbol(AMQPMessageSupport.JMS_DELIVERY_TIME, time);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Persister<org.apache.activemq.artemis.api.core.Message> getPersister() {
|
||||
return AMQPMessagePersister.getInstance();
|
||||
}
|
||||
|
||||
private synchronized void partialDecode(ByteBuffer buffer) {
|
||||
DecoderImpl decoder = TLSEncode.getDecoder();
|
||||
decoder.setByteBuffer(buffer);
|
||||
buffer.position(0);
|
||||
|
||||
_header = null;
|
||||
_deliveryAnnotations = null;
|
||||
_messageAnnotations = null;
|
||||
_properties = null;
|
||||
applicationProperties = null;
|
||||
Section section = null;
|
||||
|
||||
try {
|
||||
if (buffer.hasRemaining()) {
|
||||
section = (Section) decoder.readObject();
|
||||
}
|
||||
|
||||
if (section instanceof Header) {
|
||||
sendFrom = buffer.position();
|
||||
_header = (Header) section;
|
||||
|
||||
if (_header.getTtl() != null) {
|
||||
this.expiration = System.currentTimeMillis() + _header.getTtl().intValue();
|
||||
}
|
||||
|
||||
if (buffer.hasRemaining()) {
|
||||
section = (Section) decoder.readObject();
|
||||
} else {
|
||||
section = null;
|
||||
}
|
||||
} else {
|
||||
// meaning there is no header
|
||||
sendFrom = 0;
|
||||
}
|
||||
if (section instanceof DeliveryAnnotations) {
|
||||
_deliveryAnnotations = (DeliveryAnnotations) section;
|
||||
sendFrom = buffer.position();
|
||||
|
||||
if (buffer.hasRemaining()) {
|
||||
section = (Section) decoder.readObject();
|
||||
} else {
|
||||
section = null;
|
||||
}
|
||||
|
||||
}
|
||||
if (section instanceof MessageAnnotations) {
|
||||
_messageAnnotations = (MessageAnnotations) section;
|
||||
|
||||
if (buffer.hasRemaining()) {
|
||||
section = (Section) decoder.readObject();
|
||||
} else {
|
||||
section = null;
|
||||
}
|
||||
|
||||
}
|
||||
if (section instanceof Properties) {
|
||||
_properties = (Properties) section;
|
||||
|
||||
if (buffer.hasRemaining()) {
|
||||
section = (Section) decoder.readObject();
|
||||
} else {
|
||||
section = null;
|
||||
}
|
||||
}
|
||||
|
||||
if (section instanceof ApplicationProperties) {
|
||||
applicationProperties = (ApplicationProperties) section;
|
||||
}
|
||||
} finally {
|
||||
decoder.setByteBuffer(null);
|
||||
}
|
||||
}
|
||||
|
||||
public long getMessageFormat() {
|
||||
return messageFormat;
|
||||
}
|
||||
|
||||
public int getLength() {
|
||||
return data.array().length;
|
||||
}
|
||||
|
||||
public byte[] getArray() {
|
||||
return data.array();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void messageChanged() {
|
||||
bufferValid = false;
|
||||
this.data = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuf getBuffer() {
|
||||
if (data == null) {
|
||||
return null;
|
||||
} else {
|
||||
return Unpooled.wrappedBuffer(data);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public AMQPMessage setBuffer(ByteBuf buffer) {
|
||||
this.data = null;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message copy() {
|
||||
checkBuffer();
|
||||
AMQPMessage newEncode = new AMQPMessage(this.messageFormat, data.array());
|
||||
return newEncode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message copy(long newID) {
|
||||
checkBuffer();
|
||||
return copy().setMessageID(newID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMessageID() {
|
||||
return messageID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message setMessageID(long id) {
|
||||
this.messageID = id;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getExpiration() {
|
||||
return expiration;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AMQPMessage setExpiration(long expiration) {
|
||||
this.expiration = expiration;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getUserID() {
|
||||
Properties properties = getProperties();
|
||||
if (properties != null && properties.getUserId() != null) {
|
||||
return properties.getUserId();
|
||||
} else {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message setUserID(Object userID) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDurable() {
|
||||
if (getHeader() != null && getHeader().getDurable() != null) {
|
||||
return getHeader().getDurable().booleanValue();
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message setDurable(boolean durable) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getAddress() {
|
||||
if (address == null) {
|
||||
Properties properties = getProtonMessage().getProperties();
|
||||
if (properties != null) {
|
||||
return properties.getTo();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
} else {
|
||||
return address;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public AMQPMessage setAddress(String address) {
|
||||
this.address = address;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AMQPMessage setAddress(SimpleString address) {
|
||||
return setAddress(address.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimpleString getAddressSimpleString() {
|
||||
return SimpleString.toSimpleString(getAddress());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTimestamp() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message setTimestamp(long timestamp) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getPriority() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message setPriority(byte priority) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receiveBuffer(ByteBuf buffer) {
|
||||
|
||||
}
|
||||
|
||||
private synchronized void checkBuffer() {
|
||||
if (!bufferValid) {
|
||||
ByteBuf buffer = PooledByteBufAllocator.DEFAULT.heapBuffer(1500);
|
||||
try {
|
||||
getProtonMessage().encode(new NettyWritable(buffer));
|
||||
byte[] bytes = new byte[buffer.writerIndex()];
|
||||
buffer.readBytes(bytes);
|
||||
this.data = Unpooled.wrappedBuffer(bytes);
|
||||
} finally {
|
||||
buffer.release();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendBuffer(ByteBuf buffer, int deliveryCount) {
|
||||
checkBuffer();
|
||||
Header header = getHeader();
|
||||
if (header == null && deliveryCount > 0) {
|
||||
header = new Header();
|
||||
}
|
||||
if (header != null) {
|
||||
synchronized (header) {
|
||||
header.setDeliveryCount(UnsignedInteger.valueOf(deliveryCount - 1));
|
||||
TLSEncode.getEncoder().setByteBuffer(new NettyWritable(buffer));
|
||||
TLSEncode.getEncoder().writeObject(header);
|
||||
TLSEncode.getEncoder().setByteBuffer((WritableBuffer)null);
|
||||
}
|
||||
}
|
||||
buffer.writeBytes(data, sendFrom, data.writerIndex() - sendFrom);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putBooleanProperty(String key, boolean value) {
|
||||
getApplicationPropertiesMap().put(key, Boolean.valueOf(value));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putByteProperty(String key, byte value) {
|
||||
getApplicationPropertiesMap().put(key, Byte.valueOf(value));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putBytesProperty(String key, byte[] value) {
|
||||
getApplicationPropertiesMap().put(key, value);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putShortProperty(String key, short value) {
|
||||
getApplicationPropertiesMap().put(key, Short.valueOf(value));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putCharProperty(String key, char value) {
|
||||
getApplicationPropertiesMap().put(key, Character.valueOf(value));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putIntProperty(String key, int value) {
|
||||
getApplicationPropertiesMap().put(key, Integer.valueOf(value));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putLongProperty(String key, long value) {
|
||||
getApplicationPropertiesMap().put(key, Long.valueOf(value));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putFloatProperty(String key, float value) {
|
||||
getApplicationPropertiesMap().put(key, Float.valueOf(value));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putDoubleProperty(String key, double value) {
|
||||
getApplicationPropertiesMap().put(key, Double.valueOf(value));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putBooleanProperty(SimpleString key, boolean value) {
|
||||
getApplicationPropertiesMap().put(key, Boolean.valueOf(value));
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putByteProperty(SimpleString key, byte value) {
|
||||
return putByteProperty(key.toString(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putBytesProperty(SimpleString key, byte[] value) {
|
||||
return putBytesProperty(key.toString(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putShortProperty(SimpleString key, short value) {
|
||||
return putShortProperty(key.toString(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putCharProperty(SimpleString key, char value) {
|
||||
return putCharProperty(key.toString(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putIntProperty(SimpleString key, int value) {
|
||||
return putIntProperty(key.toString(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putLongProperty(SimpleString key, long value) {
|
||||
return putLongProperty(key.toString(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putFloatProperty(SimpleString key, float value) {
|
||||
return putFloatProperty(key.toString(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putDoubleProperty(SimpleString key, double value) {
|
||||
return putDoubleProperty(key.toString(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putStringProperty(String key, String value) {
|
||||
getApplicationPropertiesMap().put(key, value);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putObjectProperty(String key,
|
||||
Object value) throws ActiveMQPropertyConversionException {
|
||||
getApplicationPropertiesMap().put(key, value);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putObjectProperty(SimpleString key,
|
||||
Object value) throws ActiveMQPropertyConversionException {
|
||||
return putObjectProperty(key.toString(), value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object removeProperty(String key) {
|
||||
return getApplicationPropertiesMap().remove(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsProperty(String key) {
|
||||
return getApplicationPropertiesMap().containsKey(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean getBooleanProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
return (Boolean)getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Byte getByteProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
return (Byte)getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Double getDoubleProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
return (Double)getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getIntProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
return (Integer)getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getLongProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
return (Long)getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObjectProperty(String key) {
|
||||
if (key.equals("JMSType")) {
|
||||
return getProperties().getSubject();
|
||||
}
|
||||
|
||||
return getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Short getShortProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
return (Short)getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Float getFloatProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
return (Float)getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStringProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
if (key.equals("JMSType")) {
|
||||
return getProperties().getSubject();
|
||||
}
|
||||
return (String)getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsDeliveryAnnotationProperty(SimpleString key) {
|
||||
parseHeaders();
|
||||
if (_deliveryAnnotations == null || _deliveryAnnotations.getValue() == null) {
|
||||
return false;
|
||||
}
|
||||
return _deliveryAnnotations.getValue().containsKey(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object removeDeliveryAnnoationProperty(SimpleString key) {
|
||||
parseHeaders();
|
||||
if (_deliveryAnnotations == null || _deliveryAnnotations.getValue() == null) {
|
||||
return null;
|
||||
}
|
||||
return _deliveryAnnotations.getValue().remove(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getDeliveryAnnotationProperty(SimpleString key) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimpleString getSimpleStringProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
return SimpleString.toSimpleString((String)getApplicationPropertiesMap().get(key));
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getBytesProperty(String key) throws ActiveMQPropertyConversionException {
|
||||
return (byte[]) getApplicationPropertiesMap().get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object removeProperty(SimpleString key) {
|
||||
return removeProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsProperty(SimpleString key) {
|
||||
return containsProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean getBooleanProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getBooleanProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Byte getByteProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getByteProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Double getDoubleProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getDoubleProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Integer getIntProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getIntProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getLongProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getLongProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObjectProperty(SimpleString key) {
|
||||
return getObjectProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Short getShortProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getShortProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Float getFloatProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getFloatProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getStringProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getStringProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimpleString getSimpleStringProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getSimpleStringProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getBytesProperty(SimpleString key) throws ActiveMQPropertyConversionException {
|
||||
return getBytesProperty(key.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message putStringProperty(SimpleString key, SimpleString value) {
|
||||
return putStringProperty(key.toString(), value.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getEncodeSize() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<SimpleString> getPropertyNames() {
|
||||
HashSet<SimpleString> values = new HashSet<>();
|
||||
for (Object k : getApplicationPropertiesMap().keySet()) {
|
||||
values.add(SimpleString.toSimpleString(k.toString()));
|
||||
}
|
||||
return values;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMemoryEstimate() {
|
||||
if (memoryEstimate == -1) {
|
||||
memoryEstimate = memoryOffset +
|
||||
(data != null ? data.capacity() : 0);
|
||||
}
|
||||
|
||||
return memoryEstimate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ICoreMessage toCore() {
|
||||
try {
|
||||
return AMQPConverter.getInstance().toCore(this);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public SimpleString getReplyTo() {
|
||||
if (getProperties() != null) {
|
||||
return SimpleString.toSimpleString(getProperties().getReplyTo());
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public AMQPMessage setReplyTo(SimpleString address) {
|
||||
if (getProperties() != null) {
|
||||
getProperties().setReplyTo(address != null ? address.toString() : null);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int getPersistSize() {
|
||||
checkBuffer();
|
||||
return data.array().length + DataConstants.SIZE_INT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persist(ActiveMQBuffer targetRecord) {
|
||||
checkBuffer();
|
||||
targetRecord.writeInt(data.array().length);
|
||||
targetRecord.writeBytes(data.array());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reloadPersistence(ActiveMQBuffer record) {
|
||||
int size = record.readInt();
|
||||
byte[] recordArray = new byte[size];
|
||||
record.readBytes(recordArray);
|
||||
this.data = Unpooled.wrappedBuffer(recordArray);
|
||||
this.bufferValid = true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.activemq.artemis.protocol.amqp.broker;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.MessagePersister;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
|
||||
public class AMQPMessagePersister extends MessagePersister {
|
||||
|
||||
public static AMQPMessagePersister theInstance = new AMQPMessagePersister();
|
||||
|
||||
public static AMQPMessagePersister getInstance() {
|
||||
return theInstance;
|
||||
}
|
||||
|
||||
private AMQPMessagePersister() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected byte getID() {
|
||||
return ProtonProtocolManagerFactory.ID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getEncodeSize(Message record) {
|
||||
return DataConstants.SIZE_BYTE + record.getPersistSize() +
|
||||
SimpleString.sizeofNullableString(record.getAddressSimpleString()) + DataConstants.SIZE_LONG + DataConstants.SIZE_LONG;
|
||||
}
|
||||
|
||||
|
||||
/** Sub classes must add the first short as the protocol-id */
|
||||
@Override
|
||||
public void encode(ActiveMQBuffer buffer, Message record) {
|
||||
super.encode(buffer, record);
|
||||
AMQPMessage msgEncode = (AMQPMessage)record;
|
||||
buffer.writeLong(record.getMessageID());
|
||||
buffer.writeLong(msgEncode.getMessageFormat());
|
||||
buffer.writeNullableSimpleString(record.getAddressSimpleString());
|
||||
record.persist(buffer);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Message decode(ActiveMQBuffer buffer, Message record) {
|
||||
long id = buffer.readLong();
|
||||
long format = buffer.readLong();
|
||||
SimpleString address = buffer.readNullableSimpleString();
|
||||
record = new AMQPMessage(format);
|
||||
record.reloadPersistence(buffer);
|
||||
record.setMessageID(id);
|
||||
if (address != null) {
|
||||
record.setAddress(address);
|
||||
}
|
||||
return record;
|
||||
}
|
||||
|
||||
}
|
|
@ -23,6 +23,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQQueueExistsException;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
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.core.io.IOCallback;
|
||||
|
@ -32,16 +34,13 @@ import org.apache.activemq.artemis.core.server.AddressQueryResult;
|
|||
import org.apache.activemq.artemis.core.server.BindingQueryResult;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
import org.apache.activemq.artemis.core.server.QueueQueryResult;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.ServerSession;
|
||||
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerConsumerImpl;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQConnection;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.ProtonMessageConverter;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.message.EncodedMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.CoreAmqpConverter;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPException;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPInternalErrorException;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPResourceLimitExceededException;
|
||||
|
@ -65,11 +64,9 @@ import org.apache.qpid.proton.amqp.messaging.Accepted;
|
|||
import org.apache.qpid.proton.amqp.messaging.Rejected;
|
||||
import org.apache.qpid.proton.amqp.transport.AmqpError;
|
||||
import org.apache.qpid.proton.amqp.transport.ErrorCondition;
|
||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||
import org.apache.qpid.proton.engine.Delivery;
|
||||
import org.apache.qpid.proton.engine.EndpointState;
|
||||
import org.apache.qpid.proton.engine.Receiver;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.jboss.logging.Logger;
|
||||
|
||||
public class AMQPSessionCallback implements SessionCallback {
|
||||
|
@ -298,13 +295,6 @@ public class AMQPSessionCallback implements SessionCallback {
|
|||
}
|
||||
}
|
||||
|
||||
public long encodeMessage(Object message, int deliveryCount, WritableBuffer buffer) throws Exception {
|
||||
ProtonMessageConverter converter = (ProtonMessageConverter) manager.getConverter();
|
||||
|
||||
// The Proton variant accepts a WritableBuffer to allow for a faster more direct encode.
|
||||
return (long) converter.outbound((ServerMessage) message, deliveryCount, buffer);
|
||||
}
|
||||
|
||||
public String tempQueueName() {
|
||||
return UUIDGenerator.getInstance().generateStringUUID();
|
||||
}
|
||||
|
@ -321,22 +311,22 @@ public class AMQPSessionCallback implements SessionCallback {
|
|||
}
|
||||
}
|
||||
|
||||
public void ack(Transaction transaction, Object brokerConsumer, Object message) throws Exception {
|
||||
public void ack(Transaction transaction, Object brokerConsumer, Message message) throws Exception {
|
||||
if (transaction == null) {
|
||||
transaction = serverSession.getCurrentTransaction();
|
||||
}
|
||||
recoverContext();
|
||||
try {
|
||||
((ServerConsumer) brokerConsumer).individualAcknowledge(transaction, ((ServerMessage) message).getMessageID());
|
||||
((ServerConsumer) brokerConsumer).individualAcknowledge(transaction, message.getMessageID());
|
||||
} finally {
|
||||
resetContext();
|
||||
}
|
||||
}
|
||||
|
||||
public void cancel(Object brokerConsumer, Object message, boolean updateCounts) throws Exception {
|
||||
public void cancel(Object brokerConsumer, Message message, boolean updateCounts) throws Exception {
|
||||
recoverContext();
|
||||
try {
|
||||
((ServerConsumer) brokerConsumer).individualCancel(((ServerMessage) message).getMessageID(), updateCounts);
|
||||
((ServerConsumer) brokerConsumer).individualCancel(message.getMessageID(), updateCounts);
|
||||
} finally {
|
||||
resetContext();
|
||||
}
|
||||
|
@ -351,11 +341,8 @@ public class AMQPSessionCallback implements SessionCallback {
|
|||
final Delivery delivery,
|
||||
String address,
|
||||
int messageFormat,
|
||||
ByteBuf messageEncoded) throws Exception {
|
||||
EncodedMessage encodedMessage = new EncodedMessage(messageFormat, messageEncoded.array(), messageEncoded.arrayOffset(), messageEncoded.writerIndex());
|
||||
|
||||
ServerMessage message = manager.getConverter().inbound(encodedMessage);
|
||||
//use the address on the receiver if not null, if null let's hope it was set correctly on the message
|
||||
byte[] data) throws Exception {
|
||||
AMQPMessage message = new AMQPMessage(messageFormat, data);
|
||||
if (address != null) {
|
||||
message.setAddress(new SimpleString(address));
|
||||
} else {
|
||||
|
@ -372,7 +359,7 @@ public class AMQPSessionCallback implements SessionCallback {
|
|||
|
||||
recoverContext();
|
||||
|
||||
PagingStore store = manager.getServer().getPagingManager().getPageStore(message.getAddress());
|
||||
PagingStore store = manager.getServer().getPagingManager().getPageStore(message.getAddressSimpleString());
|
||||
if (store.isRejectingMessages()) {
|
||||
// We drop pre-settled messages (and abort any associated Tx)
|
||||
if (delivery.remotelySettled()) {
|
||||
|
@ -401,12 +388,12 @@ public class AMQPSessionCallback implements SessionCallback {
|
|||
}
|
||||
|
||||
private void serverSend(final Transaction transaction,
|
||||
final ServerMessage message,
|
||||
final Message message,
|
||||
final Delivery delivery,
|
||||
final Receiver receiver) throws Exception {
|
||||
try {
|
||||
|
||||
message.putStringProperty(ActiveMQConnection.CONNECTION_ID_PROPERTY_NAME.toString(), receiver.getSession().getConnection().getRemoteContainer());
|
||||
// message.putStringProperty(ActiveMQConnection.CONNECTION_ID_PROPERTY_NAME.toString(), receiver.getSession().getConnection().getRemoteContainer());
|
||||
serverSession.send(transaction, message, false, false);
|
||||
|
||||
// FIXME Potential race here...
|
||||
|
@ -416,8 +403,8 @@ public class AMQPSessionCallback implements SessionCallback {
|
|||
synchronized (connection.getLock()) {
|
||||
delivery.disposition(Accepted.getInstance());
|
||||
delivery.settle();
|
||||
connection.flush();
|
||||
}
|
||||
connection.flush(true);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -492,14 +479,14 @@ public class AMQPSessionCallback implements SessionCallback {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int sendMessage(MessageReference ref, ServerMessage message, ServerConsumer consumer, int deliveryCount) {
|
||||
public int sendMessage(MessageReference ref, Message message, ServerConsumer consumer, int deliveryCount) {
|
||||
|
||||
message.removeProperty(ActiveMQConnection.CONNECTION_ID_PROPERTY_NAME.toString());
|
||||
|
||||
ProtonServerSenderContext plugSender = (ProtonServerSenderContext) consumer.getProtocolContext();
|
||||
|
||||
try {
|
||||
return plugSender.deliverMessage(message, deliveryCount);
|
||||
return plugSender.deliverMessage(CoreAmqpConverter.checkAMQP(message), deliveryCount);
|
||||
} catch (Exception e) {
|
||||
synchronized (connection.getLock()) {
|
||||
plugSender.getSender().setCondition(new ErrorCondition(AmqpError.INTERNAL_ERROR, e.getMessage()));
|
||||
|
@ -512,7 +499,7 @@ public class AMQPSessionCallback implements SessionCallback {
|
|||
|
||||
@Override
|
||||
public int sendLargeMessage(MessageReference ref,
|
||||
ServerMessage message,
|
||||
Message message,
|
||||
ServerConsumer consumer,
|
||||
long bodySize,
|
||||
int deliveryCount) {
|
||||
|
|
|
@ -26,19 +26,17 @@ import io.netty.channel.ChannelPipeline;
|
|||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.BaseInterceptor;
|
||||
import org.apache.activemq.artemis.api.core.Interceptor;
|
||||
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.core.remoting.impl.netty.NettyServerConnection;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.core.server.management.Notification;
|
||||
import org.apache.activemq.artemis.core.server.management.NotificationListener;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.ProtonMessageConverter;
|
||||
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPConnectionContext;
|
||||
import org.apache.activemq.artemis.protocol.amqp.proton.AMQPConstants;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.ConnectionEntry;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.MessageConverter;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManagerFactory;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
|
@ -54,8 +52,6 @@ public class ProtonProtocolManager implements ProtocolManager<Interceptor>, Noti
|
|||
|
||||
private final ActiveMQServer server;
|
||||
|
||||
private MessageConverter protonConverter;
|
||||
|
||||
private final ProtonProtocolManagerFactory factory;
|
||||
|
||||
private final Map<SimpleString, RoutingType> prefixes = new HashMap<>();
|
||||
|
@ -72,18 +68,12 @@ public class ProtonProtocolManager implements ProtocolManager<Interceptor>, Noti
|
|||
public ProtonProtocolManager(ProtonProtocolManagerFactory factory, ActiveMQServer server) {
|
||||
this.factory = factory;
|
||||
this.server = server;
|
||||
this.protonConverter = new ProtonMessageConverter(server.getStorageManager());
|
||||
}
|
||||
|
||||
public ActiveMQServer getServer() {
|
||||
return server;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MessageConverter getConverter() {
|
||||
return protonConverter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNotification(Notification notification) {
|
||||
|
||||
|
|
|
@ -22,6 +22,8 @@ import java.util.Map;
|
|||
|
||||
import org.apache.activemq.artemis.api.core.BaseInterceptor;
|
||||
import org.apache.activemq.artemis.api.core.Interceptor;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.AbstractProtocolManagerFactory;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.ProtocolManager;
|
||||
|
@ -32,12 +34,24 @@ import org.osgi.service.component.annotations.Component;
|
|||
@Component(service = ProtocolManagerFactory.class)
|
||||
public class ProtonProtocolManagerFactory extends AbstractProtocolManagerFactory<Interceptor> {
|
||||
|
||||
public static final byte ID = 2;
|
||||
|
||||
private static final String AMQP_PROTOCOL_NAME = "AMQP";
|
||||
|
||||
private static final String MODULE_NAME = "artemis-amqp-protocol";
|
||||
|
||||
private static String[] SUPPORTED_PROTOCOLS = {AMQP_PROTOCOL_NAME};
|
||||
|
||||
@Override
|
||||
public byte getStoreID() {
|
||||
return ID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Persister<Message> getPersister() {
|
||||
return AMQPMessagePersister.getInstance();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProtocolManager createProtocolManager(ActiveMQServer server,
|
||||
final Map<String, Object> parameters,
|
||||
|
|
|
@ -14,7 +14,7 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.IllegalCharsetNameException;
|
|
@ -14,31 +14,31 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.MessageConverter;
|
||||
|
||||
public class AMQPNativeInboundTransformer extends AMQPRawInboundTransformer {
|
||||
|
||||
public AMQPNativeInboundTransformer(IDGenerator idGenerator) {
|
||||
super(idGenerator);
|
||||
public class AMQPConverter implements MessageConverter<AMQPMessage> {
|
||||
|
||||
private static final AMQPConverter theInstance = new AMQPConverter();
|
||||
|
||||
private AMQPConverter() {
|
||||
}
|
||||
|
||||
public static AMQPConverter getInstance() {
|
||||
return theInstance;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTransformerName() {
|
||||
return TRANSFORMER_NATIVE;
|
||||
public AMQPMessage fromCore(ICoreMessage coreMessage) throws Exception {
|
||||
return CoreAmqpConverter.fromCore(coreMessage);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InboundTransformer getFallbackTransformer() {
|
||||
return new AMQPRawInboundTransformer(idGenerator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerJMSMessage transform(EncodedMessage amqpMessage) throws Exception {
|
||||
org.apache.qpid.proton.message.Message amqp = amqpMessage.decode();
|
||||
|
||||
return populateMessage(super.transform(amqpMessage), amqp);
|
||||
public ICoreMessage toCore(AMQPMessage messageSource) throws Exception {
|
||||
return AmqpCoreConverter.toCore(messageSource);
|
||||
}
|
||||
}
|
|
@ -18,7 +18,7 @@
|
|||
* under the License.
|
||||
*
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.UUID;
|
|
@ -14,26 +14,16 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import static org.apache.activemq.artemis.api.core.Message.BYTES_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.DEFAULT_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.MAP_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.OBJECT_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.STREAM_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.TEXT_TYPE;
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||
|
||||
import javax.jms.Destination;
|
||||
import javax.jms.JMSException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import javax.jms.Destination;
|
||||
import javax.jms.JMSException;
|
||||
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ResetLimitWrappedActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerMessageImpl;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSBytesMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMapMessage;
|
||||
|
@ -42,25 +32,83 @@ import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSObjectMe
|
|||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSStreamMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSTextMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPInvalidContentTypeException;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.amqp.Symbol;
|
||||
import org.apache.qpid.proton.amqp.messaging.Data;
|
||||
import org.apache.qpid.proton.message.Message;
|
||||
|
||||
import static org.apache.activemq.artemis.api.core.Message.BYTES_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.DEFAULT_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.MAP_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.OBJECT_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.STREAM_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.TEXT_TYPE;
|
||||
|
||||
/**
|
||||
* Support class containing constant values and static methods that are used to map to / from
|
||||
* AMQP Message types being sent or received.
|
||||
*/
|
||||
public final class AMQPMessageSupport {
|
||||
|
||||
public static final String JMS_REPLY_TO_TYPE_MSG_ANNOTATION_SYMBOL_NAME = "x-opt-jms-reply-to";
|
||||
|
||||
// Message Properties used to map AMQP to JMS and back
|
||||
/**
|
||||
* Attribute used to mark the class type of JMS message that a particular message
|
||||
* instance represents, used internally by the client.
|
||||
*/
|
||||
public static final Symbol JMS_MSG_TYPE = Symbol.getSymbol("x-opt-jms-msg-type");
|
||||
|
||||
/**
|
||||
* Attribute used to mark the Application defined delivery time assigned to the message
|
||||
*/
|
||||
public static final Symbol JMS_DELIVERY_TIME = Symbol.getSymbol("x-opt-delivery-time");
|
||||
|
||||
/**
|
||||
* Value mapping for JMS_MSG_TYPE which indicates the message is a generic JMS Message
|
||||
* which has no body.
|
||||
*/
|
||||
public static final byte JMS_MESSAGE = 0;
|
||||
|
||||
/**
|
||||
* Value mapping for JMS_MSG_TYPE which indicates the message is a JMS ObjectMessage
|
||||
* which has an Object value serialized in its message body.
|
||||
*/
|
||||
public static final byte JMS_OBJECT_MESSAGE = 1;
|
||||
|
||||
/**
|
||||
* Value mapping for JMS_MSG_TYPE which indicates the message is a JMS MapMessage
|
||||
* which has an Map instance serialized in its message body.
|
||||
*/
|
||||
public static final byte JMS_MAP_MESSAGE = 2;
|
||||
|
||||
/**
|
||||
* Value mapping for JMS_MSG_TYPE which indicates the message is a JMS BytesMessage
|
||||
* which has a body that consists of raw bytes.
|
||||
*/
|
||||
public static final byte JMS_BYTES_MESSAGE = 3;
|
||||
|
||||
/**
|
||||
* Value mapping for JMS_MSG_TYPE which indicates the message is a JMS StreamMessage
|
||||
* which has a body that is a structured collection of primitives values.
|
||||
*/
|
||||
public static final byte JMS_STREAM_MESSAGE = 4;
|
||||
|
||||
/**
|
||||
* Value mapping for JMS_MSG_TYPE which indicates the message is a JMS TextMessage
|
||||
* which has a body that contains a UTF-8 encoded String.
|
||||
*/
|
||||
public static final byte JMS_TEXT_MESSAGE = 5;
|
||||
|
||||
|
||||
/**
|
||||
* Content type used to mark Data sections as containing a serialized java object.
|
||||
*/
|
||||
public static final Symbol SERIALIZED_JAVA_OBJECT_CONTENT_TYPE = Symbol.getSymbol("application/x-java-serialized-object");
|
||||
|
||||
public static final String JMS_AMQP_PREFIX = "JMS_AMQP_";
|
||||
public static final int JMS_AMQP_PREFIX_LENGTH = JMS_AMQP_PREFIX.length();
|
||||
|
||||
public static final String MESSAGE_FORMAT = "MESSAGE_FORMAT";
|
||||
public static final String ORIGINAL_ENCODING = "ORIGINAL_ENCODING";
|
||||
public static final String NATIVE = "NATIVE";
|
||||
public static final String HEADER = "HEADER";
|
||||
public static final String PROPERTIES = "PROPERTIES";
|
||||
|
@ -80,8 +128,6 @@ public final class AMQPMessageSupport {
|
|||
public static final String JMS_AMQP_HEADER_DURABLE = JMS_AMQP_PREFIX + HEADER + DURABLE;
|
||||
public static final String JMS_AMQP_HEADER_PRIORITY = JMS_AMQP_PREFIX + HEADER + PRIORITY;
|
||||
public static final String JMS_AMQP_PROPERTIES = JMS_AMQP_PREFIX + PROPERTIES;
|
||||
public static final String JMS_AMQP_ORIGINAL_ENCODING = JMS_AMQP_PREFIX + ORIGINAL_ENCODING;
|
||||
public static final String JMS_AMQP_MESSAGE_FORMAT = JMS_AMQP_PREFIX + MESSAGE_FORMAT;
|
||||
public static final String JMS_AMQP_NATIVE = JMS_AMQP_PREFIX + NATIVE;
|
||||
public static final String JMS_AMQP_FIRST_ACQUIRER = JMS_AMQP_PREFIX + FIRST_ACQUIRER;
|
||||
public static final String JMS_AMQP_CONTENT_TYPE = JMS_AMQP_PREFIX + CONTENT_TYPE;
|
||||
|
@ -105,10 +151,13 @@ public final class AMQPMessageSupport {
|
|||
public static final short AMQP_VALUE_MAP = 7;
|
||||
public static final short AMQP_VALUE_LIST = 8;
|
||||
|
||||
/**
|
||||
* Content type used to mark Data sections as containing a serialized java object.
|
||||
*/
|
||||
public static final String SERIALIZED_JAVA_OBJECT_CONTENT_TYPE = "application/x-java-serialized-object";
|
||||
public static final Symbol JMS_DEST_TYPE_MSG_ANNOTATION = getSymbol("x-opt-jms-dest");
|
||||
public static final Symbol JMS_REPLY_TO_TYPE_MSG_ANNOTATION = getSymbol("x-opt-jms-reply-to");
|
||||
|
||||
public static final byte QUEUE_TYPE = 0x00;
|
||||
public static final byte TOPIC_TYPE = 0x01;
|
||||
public static final byte TEMP_QUEUE_TYPE = 0x02;
|
||||
public static final byte TEMP_TOPIC_TYPE = 0x03;
|
||||
|
||||
/**
|
||||
* Content type used to mark Data sections as containing arbitrary bytes.
|
||||
|
@ -181,23 +230,6 @@ public final class AMQPMessageSupport {
|
|||
}
|
||||
}
|
||||
|
||||
public static ServerJMSMessage wrapMessage(int messageType, ServerMessage wrapped, int deliveryCount) {
|
||||
switch (messageType) {
|
||||
case STREAM_TYPE:
|
||||
return new ServerJMSStreamMessage(wrapped, deliveryCount);
|
||||
case BYTES_TYPE:
|
||||
return new ServerJMSBytesMessage(wrapped, deliveryCount);
|
||||
case MAP_TYPE:
|
||||
return new ServerJMSMapMessage(wrapped, deliveryCount);
|
||||
case TEXT_TYPE:
|
||||
return new ServerJMSTextMessage(wrapped, deliveryCount);
|
||||
case OBJECT_TYPE:
|
||||
return new ServerJMSObjectMessage(wrapped, deliveryCount);
|
||||
default:
|
||||
return new ServerJMSMessage(wrapped, deliveryCount);
|
||||
}
|
||||
}
|
||||
|
||||
public static String toAddress(Destination destination) {
|
||||
if (destination instanceof ActiveMQDestination) {
|
||||
return ((ActiveMQDestination) destination).getAddress();
|
||||
|
@ -205,56 +237,56 @@ public final class AMQPMessageSupport {
|
|||
return null;
|
||||
}
|
||||
|
||||
public static ServerJMSBytesMessage createBytesMessage(IDGenerator idGenerator) {
|
||||
return new ServerJMSBytesMessage(newMessage(idGenerator, BYTES_TYPE), 0);
|
||||
public static ServerJMSBytesMessage createBytesMessage(long id) {
|
||||
return new ServerJMSBytesMessage(newMessage(id, BYTES_TYPE));
|
||||
}
|
||||
|
||||
public static ServerJMSMessage createBytesMessage(IDGenerator idGenerator, byte[] array, int arrayOffset, int length) throws JMSException {
|
||||
ServerJMSBytesMessage message = createBytesMessage(idGenerator);
|
||||
public static ServerJMSBytesMessage createBytesMessage(long id, byte[] array, int arrayOffset, int length) throws JMSException {
|
||||
ServerJMSBytesMessage message = createBytesMessage(id);
|
||||
message.writeBytes(array, arrayOffset, length);
|
||||
return message;
|
||||
}
|
||||
|
||||
public static ServerJMSStreamMessage createStreamMessage(IDGenerator idGenerator) {
|
||||
return new ServerJMSStreamMessage(newMessage(idGenerator, STREAM_TYPE), 0);
|
||||
public static ServerJMSStreamMessage createStreamMessage(long id) {
|
||||
return new ServerJMSStreamMessage(newMessage(id, STREAM_TYPE));
|
||||
}
|
||||
|
||||
public static ServerJMSMessage createMessage(IDGenerator idGenerator) {
|
||||
return new ServerJMSMessage(newMessage(idGenerator, DEFAULT_TYPE), 0);
|
||||
public static ServerJMSMessage createMessage(long id) {
|
||||
return new ServerJMSMessage(newMessage(id, DEFAULT_TYPE));
|
||||
}
|
||||
|
||||
public static ServerJMSTextMessage createTextMessage(IDGenerator idGenerator) {
|
||||
return new ServerJMSTextMessage(newMessage(idGenerator, TEXT_TYPE), 0);
|
||||
public static ServerJMSTextMessage createTextMessage(long id) {
|
||||
return new ServerJMSTextMessage(newMessage(id, TEXT_TYPE));
|
||||
}
|
||||
|
||||
public static ServerJMSTextMessage createTextMessage(IDGenerator idGenerator, String text) throws JMSException {
|
||||
ServerJMSTextMessage message = createTextMessage(idGenerator);
|
||||
public static ServerJMSTextMessage createTextMessage(long id, String text) throws JMSException {
|
||||
ServerJMSTextMessage message = createTextMessage(id);
|
||||
message.setText(text);
|
||||
return message;
|
||||
}
|
||||
|
||||
public static ServerJMSObjectMessage createObjectMessage(IDGenerator idGenerator) {
|
||||
return new ServerJMSObjectMessage(newMessage(idGenerator, OBJECT_TYPE), 0);
|
||||
public static ServerJMSObjectMessage createObjectMessage(long id) {
|
||||
return new ServerJMSObjectMessage(newMessage(id, OBJECT_TYPE));
|
||||
}
|
||||
|
||||
public static ServerJMSMessage createObjectMessage(IDGenerator idGenerator, Binary serializedForm) throws JMSException {
|
||||
ServerJMSObjectMessage message = createObjectMessage(idGenerator);
|
||||
public static ServerJMSMessage createObjectMessage(long id, Binary serializedForm) throws JMSException {
|
||||
ServerJMSObjectMessage message = createObjectMessage(id);
|
||||
message.setSerializedForm(serializedForm);
|
||||
return message;
|
||||
}
|
||||
|
||||
public static ServerJMSMessage createObjectMessage(IDGenerator idGenerator, byte[] array, int offset, int length) throws JMSException {
|
||||
ServerJMSObjectMessage message = createObjectMessage(idGenerator);
|
||||
public static ServerJMSMessage createObjectMessage(long id, byte[] array, int offset, int length) throws JMSException {
|
||||
ServerJMSObjectMessage message = createObjectMessage(id);
|
||||
message.setSerializedForm(new Binary(array, offset, length));
|
||||
return message;
|
||||
}
|
||||
|
||||
public static ServerJMSMapMessage createMapMessage(IDGenerator idGenerator) {
|
||||
return new ServerJMSMapMessage(newMessage(idGenerator, MAP_TYPE), 0);
|
||||
public static ServerJMSMapMessage createMapMessage(long id) {
|
||||
return new ServerJMSMapMessage(newMessage(id, MAP_TYPE));
|
||||
}
|
||||
|
||||
public static ServerJMSMapMessage createMapMessage(IDGenerator idGenerator, Map<String, Object> content) throws JMSException {
|
||||
ServerJMSMapMessage message = createMapMessage(idGenerator);
|
||||
public static ServerJMSMapMessage createMapMessage(long id, Map<String, Object> content) throws JMSException {
|
||||
ServerJMSMapMessage message = createMapMessage(id);
|
||||
final Set<Map.Entry<String, Object>> set = content.entrySet();
|
||||
for (Map.Entry<String, Object> entry : set) {
|
||||
Object value = entry.getValue();
|
||||
|
@ -267,10 +299,10 @@ public final class AMQPMessageSupport {
|
|||
return message;
|
||||
}
|
||||
|
||||
private static ServerMessageImpl newMessage(IDGenerator idGenerator, byte messageType) {
|
||||
ServerMessageImpl message = new ServerMessageImpl(idGenerator.generateID(), 512);
|
||||
private static CoreMessage newMessage(long id, byte messageType) {
|
||||
CoreMessage message = new CoreMessage(id, 512);
|
||||
message.setType(messageType);
|
||||
((ResetLimitWrappedActiveMQBuffer) message.getBodyBuffer()).setMessage(null);
|
||||
// ((ResetLimitWrappedActiveMQBuffer) message.getBodyBuffer()).setMessage(null);
|
||||
return message;
|
||||
}
|
||||
}
|
|
@ -1,12 +1,12 @@
|
|||
/*
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* 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
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* 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,
|
||||
|
@ -14,30 +14,29 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import static org.apache.activemq.artemis.api.core.Message.HDR_SCHEDULED_DELIVERY_TIME;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_CONTENT_ENCODING;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_CONTENT_TYPE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_FIRST_ACQUIRER;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_FOOTER_PREFIX;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_HEADER;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_HEADER_DURABLE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_HEADER_PRIORITY;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_MESSAGE_ANNOTATION_PREFIX;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_REPLYTO_GROUP_ID;
|
||||
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||
|
||||
import javax.jms.DeliveryMode;
|
||||
import javax.jms.JMSException;
|
||||
import javax.jms.Message;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.CharBuffer;
|
||||
import java.nio.charset.CharacterCodingException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerDestination;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSStreamMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.TLSEncode;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.amqp.Decimal128;
|
||||
import org.apache.qpid.proton.amqp.Decimal32;
|
||||
|
@ -47,32 +46,139 @@ import org.apache.qpid.proton.amqp.UnsignedByte;
|
|||
import org.apache.qpid.proton.amqp.UnsignedInteger;
|
||||
import org.apache.qpid.proton.amqp.UnsignedLong;
|
||||
import org.apache.qpid.proton.amqp.UnsignedShort;
|
||||
import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
|
||||
import org.apache.qpid.proton.amqp.messaging.AmqpValue;
|
||||
import org.apache.qpid.proton.amqp.messaging.ApplicationProperties;
|
||||
import org.apache.qpid.proton.amqp.messaging.Data;
|
||||
import org.apache.qpid.proton.amqp.messaging.Footer;
|
||||
import org.apache.qpid.proton.amqp.messaging.Header;
|
||||
import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
|
||||
import org.apache.qpid.proton.amqp.messaging.Properties;
|
||||
import org.apache.qpid.proton.amqp.messaging.Section;
|
||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||
|
||||
public abstract class InboundTransformer {
|
||||
import static org.apache.activemq.artemis.api.core.Message.HDR_SCHEDULED_DELIVERY_TIME;
|
||||
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_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_REPLYTO_GROUP_ID;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.OCTET_STREAM_CONTENT_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.createBytesMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.createMapMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.createMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.createObjectMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.createStreamMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.createTextMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.getCharsetForTextualContent;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.isContentType;
|
||||
|
||||
protected IDGenerator idGenerator;
|
||||
/**
|
||||
* This class was created just to separate concerns on AMQPConverter.
|
||||
* For better organization of the code.
|
||||
* */
|
||||
public class AmqpCoreConverter {
|
||||
|
||||
public static final String TRANSFORMER_NATIVE = "native";
|
||||
public static final String TRANSFORMER_RAW = "raw";
|
||||
public static final String TRANSFORMER_JMS = "jms";
|
||||
public static ICoreMessage toCore(AMQPMessage message) throws Exception {
|
||||
|
||||
public InboundTransformer(IDGenerator idGenerator) {
|
||||
this.idGenerator = idGenerator;
|
||||
Section body = message.getProtonMessage().getBody();
|
||||
ServerJMSMessage result;
|
||||
|
||||
if (body == null) {
|
||||
if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE.toString(), message.getProtonMessage())) {
|
||||
result = createObjectMessage(message.getMessageID());
|
||||
} else if (isContentType(OCTET_STREAM_CONTENT_TYPE, message.getProtonMessage()) || isContentType(null, message.getProtonMessage())) {
|
||||
result = createBytesMessage(message.getMessageID());
|
||||
} else {
|
||||
Charset charset = getCharsetForTextualContent(message.getProtonMessage().getContentType());
|
||||
if (charset != null) {
|
||||
result = createTextMessage(message.getMessageID());
|
||||
} else {
|
||||
result = createMessage(message.getMessageID());
|
||||
}
|
||||
}
|
||||
} else if (body instanceof Data) {
|
||||
Binary payload = ((Data) body).getValue();
|
||||
|
||||
if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE.toString(), message.getProtonMessage())) {
|
||||
result = createObjectMessage(message.getMessageID(), payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
} else if (isContentType(OCTET_STREAM_CONTENT_TYPE, message.getProtonMessage())) {
|
||||
result = createBytesMessage(message.getMessageID(), payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
} else {
|
||||
Charset charset = getCharsetForTextualContent(message.getProtonMessage().getContentType());
|
||||
if (StandardCharsets.UTF_8.equals(charset)) {
|
||||
ByteBuffer buf = ByteBuffer.wrap(payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
|
||||
try {
|
||||
CharBuffer chars = charset.newDecoder().decode(buf);
|
||||
result = createTextMessage(message.getMessageID(), String.valueOf(chars));
|
||||
} catch (CharacterCodingException e) {
|
||||
result = createBytesMessage(message.getMessageID(), payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
}
|
||||
} else {
|
||||
result = createBytesMessage(message.getMessageID(), payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
}
|
||||
}
|
||||
|
||||
} else if (body instanceof AmqpSequence) {
|
||||
AmqpSequence sequence = (AmqpSequence) body;
|
||||
ServerJMSStreamMessage m = createStreamMessage(message.getMessageID());
|
||||
for (Object item : sequence.getValue()) {
|
||||
m.writeObject(item);
|
||||
}
|
||||
|
||||
result = m;
|
||||
} else if (body instanceof AmqpValue) {
|
||||
Object value = ((AmqpValue) body).getValue();
|
||||
if (value == null || value instanceof String) {
|
||||
result = createTextMessage(message.getMessageID(), (String) value);
|
||||
|
||||
} else if (value instanceof Binary) {
|
||||
Binary payload = (Binary) value;
|
||||
|
||||
if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE.toString(), message.getProtonMessage())) {
|
||||
result = createObjectMessage(message.getMessageID(), payload);
|
||||
} else {
|
||||
result = createBytesMessage(message.getMessageID(), payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
}
|
||||
|
||||
} else if (value instanceof List) {
|
||||
ServerJMSStreamMessage m = createStreamMessage(message.getMessageID());
|
||||
for (Object item : (List<Object>) value) {
|
||||
m.writeObject(item);
|
||||
}
|
||||
result = m;
|
||||
} else if (value instanceof Map) {
|
||||
result = createMapMessage(message.getMessageID(), (Map<String, Object>) value);
|
||||
} else {
|
||||
ByteBuf buf = PooledByteBufAllocator.DEFAULT.heapBuffer(1024);
|
||||
try {
|
||||
TLSEncode.getEncoder().setByteBuffer(new NettyWritable(buf));
|
||||
TLSEncode.getEncoder().writeObject(body);
|
||||
result = createBytesMessage(message.getMessageID(), buf.array(), 0, buf.writerIndex());
|
||||
} finally {
|
||||
buf.release();
|
||||
TLSEncode.getEncoder().setByteBuffer((WritableBuffer)null);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
throw new RuntimeException("Unexpected body type: " + body.getClass());
|
||||
}
|
||||
|
||||
populateMessage(result, message.getProtonMessage());
|
||||
result.getInnerMessage().setReplyTo(message.getReplyTo());
|
||||
|
||||
result.encode();
|
||||
|
||||
return result != null ? result.getInnerMessage() : null;
|
||||
}
|
||||
|
||||
public abstract ServerJMSMessage transform(EncodedMessage amqpMessage) throws Exception;
|
||||
|
||||
public abstract String getTransformerName();
|
||||
|
||||
public abstract InboundTransformer getFallbackTransformer();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected ServerJMSMessage populateMessage(ServerJMSMessage jms, org.apache.qpid.proton.message.Message amqp) throws Exception {
|
||||
protected static ServerJMSMessage populateMessage(ServerJMSMessage jms, org.apache.qpid.proton.message.Message amqp) throws Exception {
|
||||
Header header = amqp.getHeader();
|
||||
if (header != null) {
|
||||
jms.setBooleanProperty(JMS_AMQP_HEADER, true);
|
||||
|
@ -88,7 +194,7 @@ public abstract class InboundTransformer {
|
|||
jms.setBooleanProperty(JMS_AMQP_HEADER_PRIORITY, true);
|
||||
jms.setJMSPriority(header.getPriority().intValue());
|
||||
} else {
|
||||
jms.setJMSPriority(Message.DEFAULT_PRIORITY);
|
||||
jms.setJMSPriority(javax.jms.Message.DEFAULT_PRIORITY);
|
||||
}
|
||||
|
||||
if (header.getFirstAcquirer() != null) {
|
||||
|
@ -101,7 +207,7 @@ public abstract class InboundTransformer {
|
|||
jms.setLongProperty("JMSXDeliveryCount", header.getDeliveryCount().longValue() + 1);
|
||||
}
|
||||
} else {
|
||||
jms.setJMSPriority((byte) Message.DEFAULT_PRIORITY);
|
||||
jms.setJMSPriority((byte) javax.jms.Message.DEFAULT_PRIORITY);
|
||||
jms.setJMSDeliveryMode(DeliveryMode.NON_PERSISTENT);
|
||||
}
|
||||
|
||||
|
@ -178,7 +284,7 @@ public abstract class InboundTransformer {
|
|||
// If the jms expiration has not yet been set...
|
||||
if (header != null && jms.getJMSExpiration() == 0) {
|
||||
// Then lets try to set it based on the message ttl.
|
||||
long ttl = Message.DEFAULT_TIME_TO_LIVE;
|
||||
long ttl = javax.jms.Message.DEFAULT_TIME_TO_LIVE;
|
||||
if (header.getTtl() != null) {
|
||||
ttl = header.getTtl().longValue();
|
||||
}
|
||||
|
@ -201,7 +307,7 @@ public abstract class InboundTransformer {
|
|||
return jms;
|
||||
}
|
||||
|
||||
private void setProperty(Message msg, String key, Object value) throws JMSException {
|
||||
private static void setProperty(javax.jms.Message msg, String key, Object value) throws JMSException {
|
||||
if (value instanceof UnsignedLong) {
|
||||
long v = ((UnsignedLong) value).longValue();
|
||||
msg.setLongProperty(key, v);
|
||||
|
@ -240,4 +346,6 @@ public abstract class InboundTransformer {
|
|||
msg.setObjectProperty(key, value);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
/*
|
||||
/**
|
||||
* 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.
|
||||
|
@ -6,7 +6,7 @@
|
|||
* (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
|
||||
* 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,
|
||||
|
@ -14,37 +14,17 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
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.message.AMQPMessageSupport.AMQP_DATA;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_NULL;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_SEQUENCE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_UNKNOWN;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_VALUE_BINARY;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_VALUE_LIST;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_VALUE_STRING;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.EMPTY_BINARY;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_CONTENT_ENCODING;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_CONTENT_TYPE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_DELIVERY_ANNOTATION_PREFIX;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_FIRST_ACQUIRER;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_FOOTER_PREFIX;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_HEADER;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_HEADER_DURABLE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_HEADER_PRIORITY;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_MESSAGE_ANNOTATION_PREFIX;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_MESSAGE_FORMAT;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_NATIVE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_ORIGINAL_ENCODING;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_PREFIX;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_PROPERTIES;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_REPLYTO_GROUP_ID;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.toAddress;
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||
|
||||
import java.io.UnsupportedEncodingException;
|
||||
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 java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Date;
|
||||
|
@ -54,17 +34,11 @@ import java.util.LinkedHashMap;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import javax.jms.Destination;
|
||||
import javax.jms.JMSException;
|
||||
import javax.jms.Message;
|
||||
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.core.message.impl.MessageInternal;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
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;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSBytesMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMapMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
|
@ -72,8 +46,9 @@ import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSObjectMe
|
|||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSStreamMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSTextMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPIllegalStateException;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.TLSEncode;
|
||||
import org.apache.activemq.artemis.reader.MessageUtil;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.amqp.Symbol;
|
||||
import org.apache.qpid.proton.amqp.UnsignedByte;
|
||||
|
@ -88,59 +63,65 @@ import org.apache.qpid.proton.amqp.messaging.Header;
|
|||
import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
|
||||
import org.apache.qpid.proton.amqp.messaging.Properties;
|
||||
import org.apache.qpid.proton.amqp.messaging.Section;
|
||||
import org.apache.qpid.proton.codec.AMQPDefinedTypes;
|
||||
import org.apache.qpid.proton.codec.DecoderImpl;
|
||||
import org.apache.qpid.proton.codec.EncoderImpl;
|
||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||
import org.jboss.logging.Logger;
|
||||
|
||||
public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
||||
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;
|
||||
|
||||
private static final Logger logger = Logger.getLogger(JMSMappingOutboundTransformer.class);
|
||||
public class CoreAmqpConverter {
|
||||
|
||||
public static final Symbol JMS_DEST_TYPE_MSG_ANNOTATION = Symbol.valueOf("x-opt-jms-dest");
|
||||
public static final Symbol JMS_REPLY_TO_TYPE_MSG_ANNOTATION = Symbol.valueOf("x-opt-jms-reply-to");
|
||||
private static Logger logger = Logger.getLogger(CoreAmqpConverter.class);
|
||||
|
||||
public static final byte QUEUE_TYPE = 0x00;
|
||||
public static final byte TOPIC_TYPE = 0x01;
|
||||
public static final byte TEMP_QUEUE_TYPE = 0x02;
|
||||
public static final byte TEMP_TOPIC_TYPE = 0x03;
|
||||
|
||||
// For now Proton requires that we create a decoder to create an encoder
|
||||
private static class EncoderDecoderPair {
|
||||
DecoderImpl decoder = new DecoderImpl();
|
||||
EncoderImpl encoder = new EncoderImpl(decoder);
|
||||
{
|
||||
AMQPDefinedTypes.registerAllTypes(decoder, encoder);
|
||||
public static AMQPMessage checkAMQP(Message message) throws Exception {
|
||||
if (message instanceof AMQPMessage) {
|
||||
return (AMQPMessage)message;
|
||||
} else {
|
||||
// It will first convert to Core, then to AMQP
|
||||
return fromCore(message.toCore());
|
||||
}
|
||||
}
|
||||
|
||||
private static final ThreadLocal<EncoderDecoderPair> tlsCodec = new ThreadLocal<EncoderDecoderPair>() {
|
||||
@Override
|
||||
protected EncoderDecoderPair initialValue() {
|
||||
return new EncoderDecoderPair();
|
||||
public static AMQPMessage fromCore(ICoreMessage coreMessage) throws Exception {
|
||||
if (coreMessage == null) {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
public JMSMappingOutboundTransformer(IDGenerator idGenerator) {
|
||||
super(idGenerator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long transform(ServerJMSMessage message, WritableBuffer buffer) throws JMSException, UnsupportedEncodingException {
|
||||
if (message == null) {
|
||||
return 0;
|
||||
}
|
||||
ServerJMSMessage message = ServerJMSMessage.wrapCoreMessage(coreMessage);
|
||||
message.decode();
|
||||
|
||||
long messageFormat = 0;
|
||||
Header header = null;
|
||||
Properties properties = null;
|
||||
final Properties properties = new Properties();
|
||||
Map<Symbol, Object> daMap = null;
|
||||
Map<Symbol, Object> maMap = null;
|
||||
final Map<Symbol, Object> maMap = new HashMap<>();
|
||||
Map<String, Object> apMap = null;
|
||||
Map<Object, Object> footerMap = null;
|
||||
|
||||
Section body = convertBody(message);
|
||||
Section body = convertBody(message, maMap, properties);
|
||||
|
||||
if (message.getInnerMessage().isDurable()) {
|
||||
if (header == null) {
|
||||
|
@ -149,7 +130,7 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
header.setDurable(true);
|
||||
}
|
||||
byte priority = (byte) message.getJMSPriority();
|
||||
if (priority != Message.DEFAULT_PRIORITY) {
|
||||
if (priority != javax.jms.Message.DEFAULT_PRIORITY) {
|
||||
if (header == null) {
|
||||
header = new Header();
|
||||
}
|
||||
|
@ -157,16 +138,10 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
}
|
||||
String type = message.getJMSType();
|
||||
if (type != null) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setSubject(type);
|
||||
}
|
||||
String messageId = message.getJMSMessageID();
|
||||
if (messageId != null) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
try {
|
||||
properties.setMessageId(AMQPMessageIdHelper.INSTANCE.toIdObject(messageId));
|
||||
} catch (ActiveMQAMQPIllegalStateException e) {
|
||||
|
@ -175,31 +150,16 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
}
|
||||
Destination destination = message.getJMSDestination();
|
||||
if (destination != null) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setTo(toAddress(destination));
|
||||
if (maMap == null) {
|
||||
maMap = new HashMap<>();
|
||||
}
|
||||
maMap.put(JMS_DEST_TYPE_MSG_ANNOTATION, destinationType(destination));
|
||||
}
|
||||
Destination replyTo = message.getJMSReplyTo();
|
||||
if (replyTo != null) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setReplyTo(toAddress(replyTo));
|
||||
if (maMap == null) {
|
||||
maMap = new HashMap<>();
|
||||
}
|
||||
maMap.put(JMS_REPLY_TO_TYPE_MSG_ANNOTATION, destinationType(replyTo));
|
||||
}
|
||||
String correlationId = message.getJMSCorrelationID();
|
||||
if (correlationId != null) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
try {
|
||||
properties.setCorrelationId(AMQPMessageIdHelper.INSTANCE.toIdObject(correlationId));
|
||||
} catch (ActiveMQAMQPIllegalStateException e) {
|
||||
|
@ -218,64 +178,32 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
}
|
||||
header.setTtl(new UnsignedInteger((int) ttl));
|
||||
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setAbsoluteExpiryTime(new Date(expiration));
|
||||
}
|
||||
long timeStamp = message.getJMSTimestamp();
|
||||
if (timeStamp != 0) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setCreationTime(new Date(timeStamp));
|
||||
}
|
||||
|
||||
final Set<String> keySet = MessageUtil.getPropertyNames(message.getInnerMessage());
|
||||
for (String key : keySet) {
|
||||
if (key.startsWith("JMSX")) {
|
||||
if (key.equals("JMSXDeliveryCount")) {
|
||||
// The AMQP delivery-count field only includes prior failed delivery attempts,
|
||||
// whereas JMSXDeliveryCount includes the first/current delivery attempt.
|
||||
int amqpDeliveryCount = message.getDeliveryCount() - 1;
|
||||
if (amqpDeliveryCount > 0) {
|
||||
if (header == null) {
|
||||
header = new Header();
|
||||
}
|
||||
header.setDeliveryCount(new UnsignedInteger(amqpDeliveryCount));
|
||||
}
|
||||
continue;
|
||||
} else if (key.equals("JMSXUserID")) {
|
||||
if (key.equals("JMSXUserID")) {
|
||||
String value = message.getStringProperty(key);
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setUserId(new Binary(value.getBytes(StandardCharsets.UTF_8)));
|
||||
continue;
|
||||
} else if (key.equals("JMSXGroupID")) {
|
||||
String value = message.getStringProperty(key);
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setGroupId(value);
|
||||
continue;
|
||||
} else if (key.equals("JMSXGroupSeq")) {
|
||||
UnsignedInteger value = new UnsignedInteger(message.getIntProperty(key));
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setGroupSequence(value);
|
||||
continue;
|
||||
}
|
||||
} else if (key.startsWith(JMS_AMQP_PREFIX)) {
|
||||
// AMQP Message Information stored from a conversion to the Core Message
|
||||
if (key.equals(JMS_AMQP_MESSAGE_FORMAT)) {
|
||||
messageFormat = message.getLongProperty(JMS_AMQP_MESSAGE_FORMAT);
|
||||
continue;
|
||||
} else if (key.equals(JMS_AMQP_NATIVE)) {
|
||||
// skip..internal use only
|
||||
continue;
|
||||
} else if (key.equals(JMS_AMQP_ORIGINAL_ENCODING)) {
|
||||
if (key.equals(JMS_AMQP_NATIVE)) {
|
||||
// skip..internal use only
|
||||
continue;
|
||||
} else if (key.equals(JMS_AMQP_FIRST_ACQUIRER)) {
|
||||
|
@ -302,9 +230,6 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
header.setPriority(UnsignedByte.valueOf(priority));
|
||||
continue;
|
||||
} else if (key.startsWith(JMS_AMQP_PROPERTIES)) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
continue;
|
||||
} else if (key.startsWith(JMS_AMQP_DELIVERY_ANNOTATION_PREFIX)) {
|
||||
if (daMap == null) {
|
||||
|
@ -314,28 +239,16 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
daMap.put(Symbol.valueOf(name), message.getObjectProperty(key));
|
||||
continue;
|
||||
} else if (key.startsWith(JMS_AMQP_MESSAGE_ANNOTATION_PREFIX)) {
|
||||
if (maMap == null) {
|
||||
maMap = new HashMap<>();
|
||||
}
|
||||
String name = key.substring(JMS_AMQP_MESSAGE_ANNOTATION_PREFIX.length());
|
||||
maMap.put(Symbol.valueOf(name), message.getObjectProperty(key));
|
||||
continue;
|
||||
} else if (key.equals(JMS_AMQP_CONTENT_TYPE)) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setContentType(Symbol.getSymbol(message.getStringProperty(key)));
|
||||
continue;
|
||||
} else if (key.equals(JMS_AMQP_CONTENT_ENCODING)) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setContentEncoding(Symbol.getSymbol(message.getStringProperty(key)));
|
||||
continue;
|
||||
} else if (key.equals(JMS_AMQP_REPLYTO_GROUP_ID)) {
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setReplyToGroupId(message.getStringProperty(key));
|
||||
continue;
|
||||
} else if (key.startsWith(JMS_AMQP_FOOTER_PREFIX)) {
|
||||
|
@ -348,9 +261,6 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
}
|
||||
} else if (key.equals("_AMQ_GROUP_ID")) {
|
||||
String value = message.getStringProperty(key);
|
||||
if (properties == null) {
|
||||
properties = new Properties();
|
||||
}
|
||||
properties.setGroupId(value);
|
||||
continue;
|
||||
} else if (key.equals(NATIVE_MESSAGE_ID)) {
|
||||
|
@ -359,9 +269,6 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
} else if (key.endsWith(HDR_SCHEDULED_DELIVERY_TIME.toString())) {
|
||||
// skip..remove annotation from previous inbound transformation
|
||||
continue;
|
||||
} else if (key.equals(AMQPMessageTypes.AMQP_TYPE_KEY)) {
|
||||
// skip..internal use only - TODO - Remove this deprecated value in future release.
|
||||
continue;
|
||||
}
|
||||
|
||||
if (apMap == null) {
|
||||
|
@ -376,80 +283,69 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
apMap.put(key, objectProperty);
|
||||
}
|
||||
|
||||
EncoderImpl encoder = tlsCodec.get().encoder;
|
||||
encoder.setByteBuffer(buffer);
|
||||
|
||||
if (header != null) {
|
||||
encoder.writeObject(header);
|
||||
}
|
||||
if (daMap != null) {
|
||||
encoder.writeObject(new DeliveryAnnotations(daMap));
|
||||
}
|
||||
if (maMap != null) {
|
||||
encoder.writeObject(new MessageAnnotations(maMap));
|
||||
}
|
||||
if (properties != null) {
|
||||
encoder.writeObject(properties);
|
||||
}
|
||||
if (apMap != null) {
|
||||
encoder.writeObject(new ApplicationProperties(apMap));
|
||||
}
|
||||
if (body != null) {
|
||||
encoder.writeObject(body);
|
||||
}
|
||||
if (footerMap != null) {
|
||||
encoder.writeObject(new Footer(footerMap));
|
||||
}
|
||||
|
||||
return messageFormat;
|
||||
}
|
||||
|
||||
private Section convertBody(ServerJMSMessage message) throws JMSException {
|
||||
|
||||
Section body = null;
|
||||
short orignalEncoding = AMQP_UNKNOWN;
|
||||
ByteBuf buffer = PooledByteBufAllocator.DEFAULT.heapBuffer(1024);
|
||||
|
||||
try {
|
||||
orignalEncoding = message.getShortProperty(JMS_AMQP_ORIGINAL_ENCODING);
|
||||
} catch (Exception ex) {
|
||||
// Ignore and stick with UNKNOWN
|
||||
EncoderImpl encoder = TLSEncode.getEncoder();
|
||||
encoder.setByteBuffer(new NettyWritable(buffer));
|
||||
|
||||
if (header != null) {
|
||||
encoder.writeObject(header);
|
||||
}
|
||||
if (daMap != null) {
|
||||
encoder.writeObject(new DeliveryAnnotations(daMap));
|
||||
}
|
||||
if (maMap != null) {
|
||||
encoder.writeObject(new MessageAnnotations(maMap));
|
||||
}
|
||||
if (properties != null) {
|
||||
encoder.writeObject(properties);
|
||||
}
|
||||
if (apMap != null) {
|
||||
encoder.writeObject(new ApplicationProperties(apMap));
|
||||
}
|
||||
if (body != null) {
|
||||
encoder.writeObject(body);
|
||||
}
|
||||
if (footerMap != null) {
|
||||
encoder.writeObject(new Footer(footerMap));
|
||||
}
|
||||
|
||||
byte[] data = new byte[buffer.writerIndex()];
|
||||
buffer.readBytes(data);
|
||||
|
||||
AMQPMessage amqpMessage = new AMQPMessage(messageFormat, data);
|
||||
amqpMessage.setMessageID(message.getInnerMessage().getMessageID());
|
||||
amqpMessage.setReplyTo(coreMessage.getReplyTo());
|
||||
return amqpMessage;
|
||||
|
||||
} finally {
|
||||
TLSEncode.getEncoder().setByteBuffer((WritableBuffer) null);
|
||||
buffer.release();
|
||||
}
|
||||
}
|
||||
|
||||
private static Section convertBody(ServerJMSMessage message, Map<Symbol, Object> maMap, Properties properties) throws JMSException {
|
||||
|
||||
Section body = null;
|
||||
|
||||
if (message instanceof ServerJMSBytesMessage) {
|
||||
Binary payload = getBinaryFromMessageBody((ServerJMSBytesMessage) message);
|
||||
|
||||
maMap.put(AMQPMessageSupport.JMS_MSG_TYPE, AMQPMessageSupport.JMS_BYTES_MESSAGE);
|
||||
if (payload == null) {
|
||||
payload = EMPTY_BINARY;
|
||||
}
|
||||
|
||||
switch (orignalEncoding) {
|
||||
case AMQP_NULL:
|
||||
break;
|
||||
case AMQP_VALUE_BINARY:
|
||||
body = new AmqpValue(payload);
|
||||
break;
|
||||
case AMQP_DATA:
|
||||
case AMQP_UNKNOWN:
|
||||
default:
|
||||
body = new Data(payload);
|
||||
break;
|
||||
} else {
|
||||
body = new AmqpValue(payload);
|
||||
}
|
||||
} else if (message instanceof ServerJMSTextMessage) {
|
||||
switch (orignalEncoding) {
|
||||
case AMQP_NULL:
|
||||
break;
|
||||
case AMQP_DATA:
|
||||
body = new Data(getBinaryFromMessageBody((ServerJMSTextMessage) message));
|
||||
break;
|
||||
case AMQP_VALUE_STRING:
|
||||
case AMQP_UNKNOWN:
|
||||
default:
|
||||
body = new AmqpValue(((TextMessage) message).getText());
|
||||
break;
|
||||
}
|
||||
body = new AmqpValue(((TextMessage) message).getText());
|
||||
maMap.put(AMQPMessageSupport.JMS_MSG_TYPE, AMQPMessageSupport.JMS_TEXT_MESSAGE);
|
||||
} else if (message instanceof ServerJMSMapMessage) {
|
||||
body = new AmqpValue(getMapFromMessageBody((ServerJMSMapMessage) message));
|
||||
maMap.put(AMQPMessageSupport.JMS_MSG_TYPE, AMQPMessageSupport.JMS_MAP_MESSAGE);
|
||||
} else if (message instanceof ServerJMSStreamMessage) {
|
||||
maMap.put(AMQPMessageSupport.JMS_MSG_TYPE, AMQPMessageSupport.JMS_STREAM_MESSAGE);
|
||||
ArrayList<Object> list = new ArrayList<>();
|
||||
final ServerJMSStreamMessage m = (ServerJMSStreamMessage) message;
|
||||
try {
|
||||
|
@ -459,76 +355,48 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
} catch (MessageEOFException e) {
|
||||
}
|
||||
|
||||
// Deprecated encoding markers - TODO - Remove on future release
|
||||
if (orignalEncoding == AMQP_UNKNOWN) {
|
||||
String amqpType = message.getStringProperty(AMQPMessageTypes.AMQP_TYPE_KEY);
|
||||
if (amqpType != null) {
|
||||
if (amqpType.equals(AMQPMessageTypes.AMQP_LIST)) {
|
||||
orignalEncoding = AMQP_VALUE_LIST;
|
||||
} else {
|
||||
orignalEncoding = AMQP_SEQUENCE;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
switch (orignalEncoding) {
|
||||
case AMQP_SEQUENCE:
|
||||
body = new AmqpSequence(list);
|
||||
break;
|
||||
case AMQP_VALUE_LIST:
|
||||
case AMQP_UNKNOWN:
|
||||
default:
|
||||
body = new AmqpValue(list);
|
||||
break;
|
||||
}
|
||||
body = new AmqpSequence(list);
|
||||
} else if (message instanceof ServerJMSObjectMessage) {
|
||||
properties.setContentType(AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
|
||||
maMap.put(AMQPMessageSupport.JMS_MSG_TYPE, AMQPMessageSupport.JMS_OBJECT_MESSAGE);
|
||||
Binary payload = getBinaryFromMessageBody((ServerJMSObjectMessage) message);
|
||||
|
||||
if (payload == null) {
|
||||
payload = EMPTY_BINARY;
|
||||
}
|
||||
|
||||
switch (orignalEncoding) {
|
||||
case AMQP_VALUE_BINARY:
|
||||
body = new AmqpValue(payload);
|
||||
break;
|
||||
case AMQP_DATA:
|
||||
case AMQP_UNKNOWN:
|
||||
default:
|
||||
body = new Data(payload);
|
||||
break;
|
||||
}
|
||||
body = new Data(payload);
|
||||
|
||||
// For a non-AMQP message we tag the outbound content type as containing
|
||||
// a serialized Java object so that an AMQP client has a hint as to what
|
||||
// we are sending it.
|
||||
if (!message.propertyExists(JMS_AMQP_CONTENT_TYPE)) {
|
||||
message.setStringProperty(JMS_AMQP_CONTENT_TYPE, SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
|
||||
message.setStringProperty(JMS_AMQP_CONTENT_TYPE, SERIALIZED_JAVA_OBJECT_CONTENT_TYPE.toString());
|
||||
}
|
||||
} else if (message instanceof ServerJMSMessage) {
|
||||
maMap.put(AMQPMessageSupport.JMS_MSG_TYPE, AMQPMessageSupport.JMS_MESSAGE);
|
||||
// If this is not an AMQP message that was converted then the original encoding
|
||||
// will be unknown so we check for special cases of messages with special data
|
||||
// encoded into the server message body.
|
||||
if (orignalEncoding == AMQP_UNKNOWN) {
|
||||
MessageInternal internalMessage = message.getInnerMessage();
|
||||
int readerIndex = internalMessage.getBodyBuffer().readerIndex();
|
||||
try {
|
||||
Object s = internalMessage.getBodyBuffer().readNullableSimpleString();
|
||||
if (s != null) {
|
||||
body = new AmqpValue(s.toString());
|
||||
}
|
||||
} catch (Throwable ignored) {
|
||||
logger.debug("Exception ignored during conversion, should be ok!", ignored.getMessage(), ignored);
|
||||
} finally {
|
||||
internalMessage.getBodyBuffer().readerIndex(readerIndex);
|
||||
ICoreMessage internalMessage = message.getInnerMessage();
|
||||
int readerIndex = internalMessage.getBodyBuffer().readerIndex();
|
||||
try {
|
||||
Object s = internalMessage.getBodyBuffer().readNullableSimpleString();
|
||||
if (s != null) {
|
||||
body = new AmqpValue(s.toString());
|
||||
}
|
||||
} catch (Throwable ignored) {
|
||||
logger.debug("Exception ignored during conversion", ignored.getMessage(), ignored);
|
||||
body = new AmqpValue("Conversion to AMQP error!");
|
||||
} finally {
|
||||
internalMessage.getBodyBuffer().readerIndex(readerIndex);
|
||||
}
|
||||
}
|
||||
|
||||
return body;
|
||||
}
|
||||
|
||||
private Binary getBinaryFromMessageBody(ServerJMSBytesMessage message) throws JMSException {
|
||||
private static Binary getBinaryFromMessageBody(ServerJMSBytesMessage message) throws JMSException {
|
||||
byte[] data = new byte[(int) message.getBodyLength()];
|
||||
message.readBytes(data);
|
||||
message.reset(); // Need to reset after readBytes or future readBytes
|
||||
|
@ -536,7 +404,7 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
return new Binary(data);
|
||||
}
|
||||
|
||||
private Binary getBinaryFromMessageBody(ServerJMSTextMessage message) throws JMSException {
|
||||
private static Binary getBinaryFromMessageBody(ServerJMSTextMessage message) throws JMSException {
|
||||
Binary result = null;
|
||||
String text = message.getText();
|
||||
if (text != null) {
|
||||
|
@ -546,7 +414,7 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
return result;
|
||||
}
|
||||
|
||||
private Binary getBinaryFromMessageBody(ServerJMSObjectMessage message) throws JMSException {
|
||||
private static Binary getBinaryFromMessageBody(ServerJMSObjectMessage message) throws JMSException {
|
||||
message.getInnerMessage().getBodyBuffer().resetReaderIndex();
|
||||
int size = message.getInnerMessage().getBodyBuffer().readInt();
|
||||
byte[] bytes = new byte[size];
|
||||
|
@ -555,7 +423,7 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
return new Binary(bytes);
|
||||
}
|
||||
|
||||
private Map<String, Object> getMapFromMessageBody(ServerJMSMapMessage message) throws JMSException {
|
||||
private static Map<String, Object> getMapFromMessageBody(ServerJMSMapMessage message) throws JMSException {
|
||||
final HashMap<String, Object> map = new LinkedHashMap<>();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
|
@ -589,4 +457,5 @@ public class JMSMappingOutboundTransformer extends OutboundTransformer {
|
|||
|
||||
throw new IllegalArgumentException("Unknown Destination Type passed to JMS Transformer.");
|
||||
}
|
||||
|
||||
}
|
|
@ -1,101 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_NATIVE;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import javax.jms.BytesMessage;
|
||||
|
||||
import org.apache.activemq.artemis.core.client.ActiveMQClientLogger;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSBytesMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPNativeOutboundTransformer;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.message.EncodedMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.message.InboundTransformer;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.message.JMSMappingInboundTransformer;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.message.JMSMappingOutboundTransformer;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.message.OutboundTransformer;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.MessageConverter;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
|
||||
public class ProtonMessageConverter implements MessageConverter {
|
||||
|
||||
public ProtonMessageConverter(IDGenerator idGenerator) {
|
||||
inboundTransformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
outboundTransformer = new JMSMappingOutboundTransformer(idGenerator);
|
||||
}
|
||||
|
||||
private final InboundTransformer inboundTransformer;
|
||||
private final OutboundTransformer outboundTransformer;
|
||||
|
||||
@Override
|
||||
public ServerMessage inbound(Object messageSource) throws Exception {
|
||||
EncodedMessage encodedMessageSource = (EncodedMessage) messageSource;
|
||||
ServerJMSMessage transformedMessage = null;
|
||||
|
||||
try {
|
||||
transformedMessage = inboundTransformer.transform(encodedMessageSource);
|
||||
} catch (Exception e) {
|
||||
ActiveMQClientLogger.LOGGER.debug("Transform of message using [{}] transformer, failed" + inboundTransformer.getTransformerName());
|
||||
ActiveMQClientLogger.LOGGER.trace("Transformation error:", e);
|
||||
|
||||
throw new IOException("Failed to transform incoming delivery, skipping.");
|
||||
}
|
||||
|
||||
transformedMessage.encode();
|
||||
|
||||
return (ServerMessage) transformedMessage.getInnerMessage();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object outbound(ServerMessage messageOutbound, int deliveryCount) throws Exception {
|
||||
// Useful for testing but not recommended for real life use.
|
||||
ByteBuf nettyBuffer = Unpooled.buffer(1024);
|
||||
NettyWritable buffer = new NettyWritable(nettyBuffer);
|
||||
long messageFormat = (long) outbound(messageOutbound, deliveryCount, buffer);
|
||||
|
||||
EncodedMessage encoded = new EncodedMessage(messageFormat, nettyBuffer.array(), nettyBuffer.arrayOffset() + nettyBuffer.readerIndex(),
|
||||
nettyBuffer.readableBytes());
|
||||
|
||||
return encoded;
|
||||
}
|
||||
|
||||
public Object outbound(ServerMessage messageOutbound, int deliveryCount, WritableBuffer buffer) throws Exception {
|
||||
ServerJMSMessage jmsMessage = AMQPMessageSupport.wrapMessage(messageOutbound.getType(), messageOutbound, deliveryCount);
|
||||
|
||||
jmsMessage.decode();
|
||||
|
||||
if (jmsMessage.getBooleanProperty(JMS_AMQP_NATIVE)) {
|
||||
if (jmsMessage instanceof BytesMessage) {
|
||||
return AMQPNativeOutboundTransformer.transform(outboundTransformer, (ServerJMSBytesMessage) jmsMessage, buffer);
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
} else {
|
||||
return outboundTransformer.transform(jmsMessage, buffer);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -19,8 +19,8 @@ package org.apache.activemq.artemis.protocol.amqp.converter.jms;
|
|||
import javax.jms.BytesMessage;
|
||||
import javax.jms.JMSException;
|
||||
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageImpl;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
|
||||
import static org.apache.activemq.artemis.reader.BytesMessageUtil.bytesMessageReset;
|
||||
import static org.apache.activemq.artemis.reader.BytesMessageUtil.bytesReadBoolean;
|
||||
|
@ -49,13 +49,13 @@ import static org.apache.activemq.artemis.reader.BytesMessageUtil.bytesWriteUTF;
|
|||
|
||||
public class ServerJMSBytesMessage extends ServerJMSMessage implements BytesMessage {
|
||||
|
||||
public ServerJMSBytesMessage(MessageInternal message, int deliveryCount) {
|
||||
super(message, deliveryCount);
|
||||
public ServerJMSBytesMessage(ICoreMessage message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getBodyLength() throws JMSException {
|
||||
return message.getEndOfBodyPosition() - MessageImpl.BODY_OFFSET;
|
||||
return message.getEndOfBodyPosition() - CoreMessage.BODY_OFFSET;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,9 +25,9 @@ import java.util.HashSet;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQPropertyConversionException;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.utils.TypedProperties;
|
||||
|
||||
import static org.apache.activemq.artemis.reader.MapMessageUtil.readBodyMap;
|
||||
|
@ -52,8 +52,8 @@ public final class ServerJMSMapMessage extends ServerJMSMessage implements MapMe
|
|||
/*
|
||||
* This constructor is used to construct messages prior to sending
|
||||
*/
|
||||
public ServerJMSMapMessage(MessageInternal message, int deliveryCount) {
|
||||
super(message, deliveryCount);
|
||||
public ServerJMSMapMessage(ICoreMessage message) {
|
||||
super(message);
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -16,51 +16,64 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.jms;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Enumeration;
|
||||
|
||||
import javax.jms.DeliveryMode;
|
||||
import javax.jms.Destination;
|
||||
import javax.jms.JMSException;
|
||||
import javax.jms.Message;
|
||||
import java.util.Collections;
|
||||
import java.util.Enumeration;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
|
||||
import org.apache.activemq.artemis.reader.MessageUtil;
|
||||
|
||||
import static org.apache.activemq.artemis.api.core.FilterConstants.NATIVE_MESSAGE_ID;
|
||||
import static org.apache.activemq.artemis.api.core.Message.BYTES_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.MAP_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.OBJECT_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.STREAM_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.TEXT_TYPE;
|
||||
|
||||
public class ServerJMSMessage implements Message {
|
||||
|
||||
protected final MessageInternal message;
|
||||
protected final ICoreMessage message;
|
||||
private ActiveMQBuffer readBodyBuffer;
|
||||
|
||||
protected int deliveryCount;
|
||||
public ServerJMSMessage(ICoreMessage message) {
|
||||
this.message = message;
|
||||
}
|
||||
|
||||
public MessageInternal getInnerMessage() {
|
||||
public static ServerJMSMessage wrapCoreMessage(ICoreMessage wrapped) {
|
||||
switch (wrapped.getType()) {
|
||||
case STREAM_TYPE:
|
||||
return new ServerJMSStreamMessage(wrapped);
|
||||
case BYTES_TYPE:
|
||||
return new ServerJMSBytesMessage(wrapped);
|
||||
case MAP_TYPE:
|
||||
return new ServerJMSMapMessage(wrapped);
|
||||
case TEXT_TYPE:
|
||||
return new ServerJMSTextMessage(wrapped);
|
||||
case OBJECT_TYPE:
|
||||
return new ServerJMSObjectMessage(wrapped);
|
||||
default:
|
||||
return new ServerJMSMessage(wrapped);
|
||||
}
|
||||
}
|
||||
|
||||
public ICoreMessage getInnerMessage() {
|
||||
return message;
|
||||
}
|
||||
|
||||
public ServerJMSMessage(MessageInternal message, int deliveryCount) {
|
||||
this.message = message;
|
||||
this.deliveryCount = deliveryCount;
|
||||
}
|
||||
|
||||
public int getDeliveryCount() {
|
||||
return deliveryCount;
|
||||
}
|
||||
|
||||
private ActiveMQBuffer readBodyBuffer;
|
||||
|
||||
/**
|
||||
* When reading we use a protected copy so multi-threads can work fine
|
||||
*/
|
||||
protected ActiveMQBuffer getReadBodyBuffer() {
|
||||
if (readBodyBuffer == null) {
|
||||
// to avoid clashes between multiple threads
|
||||
readBodyBuffer = message.getBodyBufferDuplicate();
|
||||
readBodyBuffer = message.getReadOnlyBodyBuffer();
|
||||
}
|
||||
return readBodyBuffer;
|
||||
}
|
||||
|
@ -113,13 +126,13 @@ public class ServerJMSMessage implements Message {
|
|||
}
|
||||
|
||||
@Override
|
||||
public final void setJMSCorrelationID(String correlationID) throws JMSException {
|
||||
MessageUtil.setJMSCorrelationID(message, correlationID);
|
||||
public final String getJMSCorrelationID() throws JMSException {
|
||||
return MessageUtil.getJMSCorrelationID(message);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final String getJMSCorrelationID() throws JMSException {
|
||||
return MessageUtil.getJMSCorrelationID(message);
|
||||
public final void setJMSCorrelationID(String correlationID) throws JMSException {
|
||||
MessageUtil.setJMSCorrelationID(message, correlationID);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -140,7 +153,7 @@ public class ServerJMSMessage implements Message {
|
|||
|
||||
@Override
|
||||
public final Destination getJMSDestination() throws JMSException {
|
||||
SimpleString sdest = message.getAddress();
|
||||
SimpleString sdest = message.getAddressSimpleString();
|
||||
|
||||
if (sdest == null) {
|
||||
return null;
|
||||
|
@ -152,7 +165,7 @@ public class ServerJMSMessage implements Message {
|
|||
@Override
|
||||
public final void setJMSDestination(Destination destination) throws JMSException {
|
||||
if (destination == null) {
|
||||
message.setAddress(null);
|
||||
message.setAddress((SimpleString)null);
|
||||
} else {
|
||||
message.setAddress(((ActiveMQDestination) destination).getSimpleAddress());
|
||||
}
|
||||
|
@ -254,19 +267,11 @@ public class ServerJMSMessage implements Message {
|
|||
|
||||
@Override
|
||||
public final int getIntProperty(String name) throws JMSException {
|
||||
if (MessageUtil.JMSXDELIVERYCOUNT.equals(name)) {
|
||||
return deliveryCount;
|
||||
}
|
||||
|
||||
return message.getIntProperty(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final long getLongProperty(String name) throws JMSException {
|
||||
if (MessageUtil.JMSXDELIVERYCOUNT.equals(name)) {
|
||||
return deliveryCount;
|
||||
}
|
||||
|
||||
return message.getLongProperty(name);
|
||||
}
|
||||
|
||||
|
@ -282,10 +287,6 @@ public class ServerJMSMessage implements Message {
|
|||
|
||||
@Override
|
||||
public final String getStringProperty(String name) throws JMSException {
|
||||
if (MessageUtil.JMSXDELIVERYCOUNT.equals(name)) {
|
||||
return String.valueOf(deliveryCount);
|
||||
}
|
||||
|
||||
return message.getStringProperty(name);
|
||||
}
|
||||
|
||||
|
|
|
@ -16,13 +16,12 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.jms;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
import javax.jms.JMSException;
|
||||
import javax.jms.ObjectMessage;
|
||||
import java.io.Serializable;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
|
||||
public class ServerJMSObjectMessage extends ServerJMSMessage implements ObjectMessage {
|
||||
|
@ -31,8 +30,8 @@ public class ServerJMSObjectMessage extends ServerJMSMessage implements ObjectMe
|
|||
|
||||
private Binary payload;
|
||||
|
||||
public ServerJMSObjectMessage(MessageInternal message, int deliveryCount) {
|
||||
super(message, deliveryCount);
|
||||
public ServerJMSObjectMessage(ICoreMessage message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,9 +21,9 @@ import javax.jms.MessageEOFException;
|
|||
import javax.jms.MessageFormatException;
|
||||
import javax.jms.StreamMessage;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.Pair;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
|
||||
import static org.apache.activemq.artemis.reader.StreamMessageUtil.streamReadBoolean;
|
||||
|
@ -44,8 +44,8 @@ public final class ServerJMSStreamMessage extends ServerJMSMessage implements St
|
|||
|
||||
private int bodyLength = 0;
|
||||
|
||||
public ServerJMSStreamMessage(MessageInternal message, int deliveryCount) {
|
||||
super(message, deliveryCount);
|
||||
public ServerJMSStreamMessage(ICoreMessage message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
// StreamMessage implementation ----------------------------------
|
||||
|
@ -180,7 +180,7 @@ public final class ServerJMSStreamMessage extends ServerJMSMessage implements St
|
|||
@Override
|
||||
public Object readObject() throws JMSException {
|
||||
|
||||
if (getReadBodyBuffer().readerIndex() >= message.getEndOfBodyPosition()) {
|
||||
if (getReadBodyBuffer().readerIndex() >= getReadBodyBuffer().writerIndex()) {
|
||||
throw new MessageEOFException("");
|
||||
}
|
||||
try {
|
||||
|
|
|
@ -19,9 +19,9 @@ package org.apache.activemq.artemis.protocol.amqp.converter.jms;
|
|||
import javax.jms.JMSException;
|
||||
import javax.jms.TextMessage;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.impl.MessageInternal;
|
||||
|
||||
import static org.apache.activemq.artemis.reader.TextMessageUtil.readBodyText;
|
||||
import static org.apache.activemq.artemis.reader.TextMessageUtil.writeBodyText;
|
||||
|
@ -49,8 +49,8 @@ public class ServerJMSTextMessage extends ServerJMSMessage implements TextMessag
|
|||
/*
|
||||
* This constructor is used to construct messages prior to sending
|
||||
*/
|
||||
public ServerJMSTextMessage(MessageInternal message, int deliveryCount) {
|
||||
super(message, deliveryCount);
|
||||
public ServerJMSTextMessage(ICoreMessage message) {
|
||||
super(message);
|
||||
|
||||
}
|
||||
// TextMessage implementation ------------------------------------
|
||||
|
|
|
@ -1,80 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import java.io.UnsupportedEncodingException;
|
||||
|
||||
import javax.jms.JMSException;
|
||||
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSBytesMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.qpid.proton.amqp.UnsignedInteger;
|
||||
import org.apache.qpid.proton.amqp.messaging.Header;
|
||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||
import org.apache.qpid.proton.message.ProtonJMessage;
|
||||
|
||||
public class AMQPNativeOutboundTransformer extends OutboundTransformer {
|
||||
|
||||
public AMQPNativeOutboundTransformer(IDGenerator idGenerator) {
|
||||
super(idGenerator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long transform(ServerJMSMessage message, WritableBuffer buffer) throws JMSException, UnsupportedEncodingException {
|
||||
if (message == null || !(message instanceof ServerJMSBytesMessage)) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
return transform(this, (ServerJMSBytesMessage) message, buffer);
|
||||
}
|
||||
|
||||
public static long transform(OutboundTransformer options, ServerJMSBytesMessage message, WritableBuffer buffer) throws JMSException {
|
||||
byte[] data = new byte[(int) message.getBodyLength()];
|
||||
message.readBytes(data);
|
||||
message.reset();
|
||||
|
||||
// The AMQP delivery-count field only includes prior failed delivery attempts,
|
||||
int amqpDeliveryCount = message.getDeliveryCount() - 1;
|
||||
if (amqpDeliveryCount >= 1) {
|
||||
|
||||
// decode...
|
||||
ProtonJMessage amqp = (ProtonJMessage) org.apache.qpid.proton.message.Message.Factory.create();
|
||||
int offset = 0;
|
||||
int len = data.length;
|
||||
while (len > 0) {
|
||||
final int decoded = amqp.decode(data, offset, len);
|
||||
assert decoded > 0 : "Make progress decoding the message";
|
||||
offset += decoded;
|
||||
len -= decoded;
|
||||
}
|
||||
|
||||
// Update the DeliveryCount header which might require adding a Header
|
||||
if (amqp.getHeader() == null && amqpDeliveryCount > 0) {
|
||||
amqp.setHeader(new Header());
|
||||
}
|
||||
|
||||
amqp.getHeader().setDeliveryCount(new UnsignedInteger(amqpDeliveryCount));
|
||||
|
||||
amqp.encode(buffer);
|
||||
} else {
|
||||
buffer.put(data, 0, data.length);
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
}
|
|
@ -1,62 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_MESSAGE_FORMAT;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_NATIVE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.createBytesMessage;
|
||||
|
||||
import javax.jms.DeliveryMode;
|
||||
import javax.jms.Message;
|
||||
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSBytesMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
|
||||
public class AMQPRawInboundTransformer extends InboundTransformer {
|
||||
|
||||
public AMQPRawInboundTransformer(IDGenerator idGenerator) {
|
||||
super(idGenerator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTransformerName() {
|
||||
return TRANSFORMER_RAW;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InboundTransformer getFallbackTransformer() {
|
||||
return null; // No fallback from full raw transform
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerJMSMessage transform(EncodedMessage amqpMessage) throws Exception {
|
||||
ServerJMSBytesMessage message = createBytesMessage(idGenerator);
|
||||
message.writeBytes(amqpMessage.getArray(), amqpMessage.getArrayOffset(), amqpMessage.getLength());
|
||||
|
||||
// We cannot decode the message headers to check so err on the side of caution
|
||||
// and mark all messages as persistent.
|
||||
message.setJMSDeliveryMode(DeliveryMode.PERSISTENT);
|
||||
message.setJMSPriority(Message.DEFAULT_PRIORITY);
|
||||
message.setJMSTimestamp(System.currentTimeMillis());
|
||||
|
||||
message.setLongProperty(JMS_AMQP_MESSAGE_FORMAT, amqpMessage.getMessageFormat());
|
||||
message.setBooleanProperty(JMS_AMQP_NATIVE, true);
|
||||
|
||||
return message;
|
||||
}
|
||||
}
|
|
@ -1,67 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.message.Message;
|
||||
|
||||
public class EncodedMessage {
|
||||
|
||||
private final Binary data;
|
||||
final long messageFormat;
|
||||
|
||||
public EncodedMessage(long messageFormat, byte[] data, int offset, int length) {
|
||||
this.data = new Binary(data, offset, length);
|
||||
this.messageFormat = messageFormat;
|
||||
}
|
||||
|
||||
public long getMessageFormat() {
|
||||
return messageFormat;
|
||||
}
|
||||
|
||||
public Message decode() throws Exception {
|
||||
Message amqp = Message.Factory.create();
|
||||
|
||||
int offset = getArrayOffset();
|
||||
int len = getLength();
|
||||
while (len > 0) {
|
||||
final int decoded = amqp.decode(getArray(), offset, len);
|
||||
assert decoded > 0 : "Make progress decoding the message";
|
||||
offset += decoded;
|
||||
len -= decoded;
|
||||
}
|
||||
|
||||
return amqp;
|
||||
}
|
||||
|
||||
public int getLength() {
|
||||
return data.getLength();
|
||||
}
|
||||
|
||||
public int getArrayOffset() {
|
||||
return data.getArrayOffset();
|
||||
}
|
||||
|
||||
public byte[] getArray() {
|
||||
return data.getArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return data.toString();
|
||||
}
|
||||
}
|
|
@ -1,196 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_DATA;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_NULL;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_SEQUENCE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_VALUE_BINARY;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_VALUE_LIST;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_VALUE_MAP;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_VALUE_NULL;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_VALUE_STRING;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_MESSAGE_FORMAT;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_ORIGINAL_ENCODING;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.OCTET_STREAM_CONTENT_TYPE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.createBytesMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.createMapMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.createMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.createObjectMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.createStreamMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.createTextMessage;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.getCharsetForTextualContent;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.isContentType;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.CharBuffer;
|
||||
import java.nio.charset.CharacterCodingException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSStreamMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPInternalErrorException;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
|
||||
import org.apache.qpid.proton.amqp.messaging.AmqpValue;
|
||||
import org.apache.qpid.proton.amqp.messaging.Data;
|
||||
import org.apache.qpid.proton.amqp.messaging.Section;
|
||||
import org.apache.qpid.proton.message.Message;
|
||||
|
||||
public class JMSMappingInboundTransformer extends InboundTransformer {
|
||||
|
||||
public JMSMappingInboundTransformer(IDGenerator idGenerator) {
|
||||
super(idGenerator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTransformerName() {
|
||||
return TRANSFORMER_JMS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InboundTransformer getFallbackTransformer() {
|
||||
return new AMQPNativeInboundTransformer(idGenerator);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerJMSMessage transform(EncodedMessage encodedMessage) throws Exception {
|
||||
ServerJMSMessage transformedMessage = null;
|
||||
|
||||
try {
|
||||
Message amqpMessage = encodedMessage.decode();
|
||||
transformedMessage = createServerMessage(amqpMessage);
|
||||
populateMessage(transformedMessage, amqpMessage);
|
||||
} catch (Exception ex) {
|
||||
InboundTransformer transformer = this.getFallbackTransformer();
|
||||
|
||||
while (transformer != null) {
|
||||
try {
|
||||
transformedMessage = transformer.transform(encodedMessage);
|
||||
break;
|
||||
} catch (Exception e) {
|
||||
transformer = transformer.getFallbackTransformer();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Regardless of the transformer that finally decoded the message we need to ensure that
|
||||
// the AMQP Message Format value is preserved for application on retransmit.
|
||||
if (transformedMessage != null && encodedMessage.getMessageFormat() != 0) {
|
||||
transformedMessage.setLongProperty(JMS_AMQP_MESSAGE_FORMAT, encodedMessage.getMessageFormat());
|
||||
}
|
||||
|
||||
return transformedMessage;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private ServerJMSMessage createServerMessage(Message message) throws Exception {
|
||||
|
||||
Section body = message.getBody();
|
||||
ServerJMSMessage result;
|
||||
|
||||
if (body == null) {
|
||||
if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE, message)) {
|
||||
result = createObjectMessage(idGenerator);
|
||||
} else if (isContentType(OCTET_STREAM_CONTENT_TYPE, message) || isContentType(null, message)) {
|
||||
result = createBytesMessage(idGenerator);
|
||||
} else {
|
||||
Charset charset = getCharsetForTextualContent(message.getContentType());
|
||||
if (charset != null) {
|
||||
result = createTextMessage(idGenerator);
|
||||
} else {
|
||||
result = createMessage(idGenerator);
|
||||
}
|
||||
}
|
||||
|
||||
result.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_NULL);
|
||||
} else if (body instanceof Data) {
|
||||
Binary payload = ((Data) body).getValue();
|
||||
|
||||
if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE, message)) {
|
||||
result = createObjectMessage(idGenerator, payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
} else if (isContentType(OCTET_STREAM_CONTENT_TYPE, message)) {
|
||||
result = createBytesMessage(idGenerator, payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
} else {
|
||||
Charset charset = getCharsetForTextualContent(message.getContentType());
|
||||
if (StandardCharsets.UTF_8.equals(charset)) {
|
||||
ByteBuffer buf = ByteBuffer.wrap(payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
|
||||
try {
|
||||
CharBuffer chars = charset.newDecoder().decode(buf);
|
||||
result = createTextMessage(idGenerator, String.valueOf(chars));
|
||||
} catch (CharacterCodingException e) {
|
||||
result = createBytesMessage(idGenerator, payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
}
|
||||
} else {
|
||||
result = createBytesMessage(idGenerator, payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
}
|
||||
}
|
||||
|
||||
result.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_DATA);
|
||||
} else if (body instanceof AmqpSequence) {
|
||||
AmqpSequence sequence = (AmqpSequence) body;
|
||||
ServerJMSStreamMessage m = createStreamMessage(idGenerator);
|
||||
for (Object item : sequence.getValue()) {
|
||||
m.writeObject(item);
|
||||
}
|
||||
|
||||
result = m;
|
||||
result.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_SEQUENCE);
|
||||
} else if (body instanceof AmqpValue) {
|
||||
Object value = ((AmqpValue) body).getValue();
|
||||
if (value == null || value instanceof String) {
|
||||
result = createTextMessage(idGenerator, (String) value);
|
||||
|
||||
result.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, value == null ? AMQP_VALUE_NULL : AMQP_VALUE_STRING);
|
||||
} else if (value instanceof Binary) {
|
||||
Binary payload = (Binary) value;
|
||||
|
||||
if (isContentType(SERIALIZED_JAVA_OBJECT_CONTENT_TYPE, message)) {
|
||||
result = createObjectMessage(idGenerator, payload);
|
||||
} else {
|
||||
result = createBytesMessage(idGenerator, payload.getArray(), payload.getArrayOffset(), payload.getLength());
|
||||
}
|
||||
|
||||
result.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
} else if (value instanceof List) {
|
||||
ServerJMSStreamMessage m = createStreamMessage(idGenerator);
|
||||
for (Object item : (List<Object>) value) {
|
||||
m.writeObject(item);
|
||||
}
|
||||
result = m;
|
||||
result.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_LIST);
|
||||
} else if (value instanceof Map) {
|
||||
result = createMapMessage(idGenerator, (Map<String, Object>) value);
|
||||
result.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_MAP);
|
||||
} else {
|
||||
// Trigger fall-back to native encoder which generates BytesMessage with the
|
||||
// original message stored in the message body.
|
||||
throw new ActiveMQAMQPInternalErrorException("Unable to encode to ActiveMQ JMS Message");
|
||||
}
|
||||
} else {
|
||||
throw new RuntimeException("Unexpected body type: " + body.getClass());
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -1,53 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import java.io.UnsupportedEncodingException;
|
||||
|
||||
import javax.jms.JMSException;
|
||||
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||
|
||||
public abstract class OutboundTransformer {
|
||||
|
||||
protected IDGenerator idGenerator;
|
||||
|
||||
public OutboundTransformer(IDGenerator idGenerator) {
|
||||
this.idGenerator = idGenerator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given an JMS Message perform a conversion to an AMQP Message and encode into a form that
|
||||
* is ready for transmission.
|
||||
*
|
||||
* @param message
|
||||
* the message to transform
|
||||
* @param buffer
|
||||
* the buffer where encoding should write to
|
||||
*
|
||||
* @return the message format key of the encoded message.
|
||||
*
|
||||
* @throws JMSException
|
||||
* if an error occurs during message transformation
|
||||
* @throws UnsupportedEncodingException
|
||||
* if an error occurs during message encoding
|
||||
*/
|
||||
public abstract long transform(ServerJMSMessage message, WritableBuffer buffer) throws JMSException, UnsupportedEncodingException;
|
||||
|
||||
}
|
|
@ -134,6 +134,10 @@ public class AMQPConnectionContext extends ProtonInitializable {
|
|||
handler.flush();
|
||||
}
|
||||
|
||||
public void flush(boolean wait) {
|
||||
handler.flush(wait);
|
||||
}
|
||||
|
||||
public void close(ErrorCondition errorCondition) {
|
||||
handler.close(errorCondition);
|
||||
}
|
||||
|
|
|
@ -134,6 +134,7 @@ public class ProtonServerReceiverContext extends ProtonInitializable implements
|
|||
@Override
|
||||
public void onMessage(Delivery delivery) throws ActiveMQAMQPException {
|
||||
Receiver receiver;
|
||||
ByteBuf buffer = null;
|
||||
try {
|
||||
receiver = ((Receiver) delivery.getLink());
|
||||
|
||||
|
@ -144,26 +145,30 @@ public class ProtonServerReceiverContext extends ProtonInitializable implements
|
|||
if (delivery.isPartial()) {
|
||||
return;
|
||||
}
|
||||
// This should be used if getDataLength was avilable
|
||||
// byte[] data = new byte[delivery.getDataLength()];
|
||||
|
||||
ByteBuf buffer = PooledByteBufAllocator.DEFAULT.heapBuffer(10 * 1024);
|
||||
try {
|
||||
synchronized (connection.getLock()) {
|
||||
DeliveryUtil.readDelivery(receiver, buffer);
|
||||
buffer = PooledByteBufAllocator.DEFAULT.heapBuffer(10 * 1024);
|
||||
Transaction tx = null;
|
||||
|
||||
receiver.advance();
|
||||
synchronized (connection.getLock()) {
|
||||
DeliveryUtil.readDelivery(receiver, buffer);
|
||||
receiver.advance();
|
||||
}
|
||||
|
||||
Transaction tx = null;
|
||||
if (delivery.getRemoteState() instanceof TransactionalState) {
|
||||
byte[] data = new byte[buffer.writerIndex()];
|
||||
buffer.readBytes(data);
|
||||
|
||||
TransactionalState txState = (TransactionalState) delivery.getRemoteState();
|
||||
tx = this.sessionSPI.getTransaction(txState.getTxnId());
|
||||
}
|
||||
sessionSPI.serverSend(tx, receiver, delivery, address, delivery.getMessageFormat(), buffer);
|
||||
if (delivery.getRemoteState() instanceof TransactionalState) {
|
||||
|
||||
flow(maxCreditAllocation, minCreditRefresh);
|
||||
}
|
||||
} finally {
|
||||
buffer.release();
|
||||
TransactionalState txState = (TransactionalState) delivery.getRemoteState();
|
||||
tx = this.sessionSPI.getTransaction(txState.getTxnId());
|
||||
}
|
||||
|
||||
sessionSPI.serverSend(tx, receiver, delivery, address, delivery.getMessageFormat(), data);
|
||||
|
||||
synchronized (connection.getLock()) {
|
||||
flow(maxCreditAllocation, minCreditRefresh);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.warn(e.getMessage(), e);
|
||||
|
@ -174,6 +179,10 @@ public class ProtonServerReceiverContext extends ProtonInitializable implements
|
|||
rejected.setError(condition);
|
||||
delivery.disposition(rejected);
|
||||
delivery.settle();
|
||||
} finally {
|
||||
if (buffer != null) {
|
||||
buffer.release();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -21,15 +21,19 @@ import java.util.Map;
|
|||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQQueueExistsException;
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQQueueExistsException;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.postoffice.impl.CompositeAddress;
|
||||
import org.apache.activemq.artemis.core.server.AddressQueryResult;
|
||||
import org.apache.activemq.artemis.core.server.QueueQueryResult;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQConnection;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPException;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPIllegalStateException;
|
||||
|
@ -38,7 +42,6 @@ import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPNotFound
|
|||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPResourceLimitExceededException;
|
||||
import org.apache.activemq.artemis.protocol.amqp.logger.ActiveMQAMQPProtocolMessageBundle;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.CreditsSemaphore;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
import org.apache.activemq.artemis.selector.filter.FilterException;
|
||||
import org.apache.activemq.artemis.selector.impl.SelectorParser;
|
||||
import org.apache.qpid.proton.amqp.DescribedType;
|
||||
|
@ -61,9 +64,6 @@ import org.apache.qpid.proton.engine.EndpointState;
|
|||
import org.apache.qpid.proton.engine.Sender;
|
||||
import org.jboss.logging.Logger;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
|
||||
/**
|
||||
* TODO: Merge {@link ProtonServerSenderContext} and {@link org.apache.activemq.artemis.protocol.amqp.client.ProtonClientSenderContext} once we support 'global' link names. The split is a workaround for outgoing links
|
||||
*/
|
||||
|
@ -474,7 +474,7 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
|||
if (closed) {
|
||||
return;
|
||||
}
|
||||
Object message = delivery.getContext();
|
||||
Message message = (Message)delivery.getContext();
|
||||
|
||||
boolean preSettle = sender.getRemoteSenderSettleMode() == SenderSettleMode.SETTLED;
|
||||
|
||||
|
@ -518,6 +518,7 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
|||
try {
|
||||
sessionSPI.ack(null, brokerConsumer, message);
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
throw ActiveMQAMQPProtocolMessageBundle.BUNDLE.errorAcknowledgingMessage(message.toString(), e.getMessage());
|
||||
}
|
||||
} else if (remoteState instanceof Released) {
|
||||
|
@ -566,7 +567,7 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
|||
/**
|
||||
* handle an out going message from ActiveMQ Artemis, send via the Proton Sender
|
||||
*/
|
||||
public int deliverMessage(Object message, int deliveryCount) throws Exception {
|
||||
public int deliverMessage(AMQPMessage message, int deliveryCount) throws Exception {
|
||||
if (closed) {
|
||||
return 0;
|
||||
}
|
||||
|
@ -590,16 +591,7 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
|||
|
||||
ByteBuf nettyBuffer = PooledByteBufAllocator.DEFAULT.heapBuffer(1024);
|
||||
try {
|
||||
long messageFormat = 0;
|
||||
|
||||
// Encode the Server Message into the given Netty Buffer as an AMQP
|
||||
// Message transformed from the internal message model.
|
||||
try {
|
||||
messageFormat = sessionSPI.encodeMessage(message, deliveryCount, new NettyWritable(nettyBuffer));
|
||||
} catch (Throwable e) {
|
||||
log.warn(e.getMessage(), e);
|
||||
throw new ActiveMQAMQPInternalErrorException(e.getMessage(), e);
|
||||
}
|
||||
message.sendBuffer(nettyBuffer, deliveryCount);
|
||||
|
||||
int size = nettyBuffer.writerIndex();
|
||||
|
||||
|
@ -609,7 +601,7 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
|||
}
|
||||
final Delivery delivery;
|
||||
delivery = sender.delivery(tag, 0, tag.length);
|
||||
delivery.setMessageFormat((int) messageFormat);
|
||||
delivery.setMessageFormat((int) message.getMessageFormat());
|
||||
delivery.setContext(message);
|
||||
|
||||
// this will avoid a copy.. patch provided by Norman using buffer.array()
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.activemq.artemis.protocol.amqp.broker.AMQPSessionCallback;
|
|||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPException;
|
||||
import org.apache.activemq.artemis.protocol.amqp.logger.ActiveMQAMQPProtocolMessageBundle;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.DeliveryUtil;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.amqp.Symbol;
|
||||
import org.apache.qpid.proton.amqp.messaging.Accepted;
|
||||
|
@ -61,7 +62,7 @@ public class ProtonTransactionHandler implements ProtonDeliveryHandler {
|
|||
return;
|
||||
}
|
||||
|
||||
DeliveryUtil.readDelivery(receiver, buffer);
|
||||
receiver.recv(new NettyWritable(buffer));
|
||||
|
||||
receiver.advance();
|
||||
|
||||
|
|
|
@ -254,7 +254,7 @@ public class ProtonHandler extends ProtonInitializable {
|
|||
flush(false);
|
||||
}
|
||||
|
||||
private void flush(boolean wait) {
|
||||
public void flush(boolean wait) {
|
||||
synchronized (lock) {
|
||||
transport.process();
|
||||
|
||||
|
|
|
@ -0,0 +1,139 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.activemq.artemis.protocol.amqp.util;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.Charset;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.qpid.proton.codec.ReadableBuffer;
|
||||
|
||||
public class NettyReadable implements ReadableBuffer {
|
||||
|
||||
private static final Charset Charset_UTF8 = Charset.forName("UTF-8");
|
||||
|
||||
private final ByteBuf buffer;
|
||||
|
||||
public NettyReadable(ByteBuf buffer) {
|
||||
this.buffer = buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(ReadableBuffer other) {
|
||||
buffer.writeBytes(other.byteBuffer());
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte get() {
|
||||
return buffer.readByte();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt() {
|
||||
return buffer.readInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong() {
|
||||
return buffer.readLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getShort() {
|
||||
return buffer.readShort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat() {
|
||||
return buffer.readFloat();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble() {
|
||||
return buffer.readDouble();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer get(byte[] data, int offset, int length) {
|
||||
buffer.readBytes(data, offset, length);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer get(byte[] data) {
|
||||
buffer.readBytes(data);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer position(int position) {
|
||||
buffer.readerIndex(position);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer slice() {
|
||||
return new NettyReadable(buffer.slice());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer flip() {
|
||||
return new NettyReadable(buffer.duplicate().setIndex(0, buffer.readerIndex()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer limit(int limit) {
|
||||
buffer.writerIndex(limit);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int limit() {
|
||||
return buffer.writerIndex();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int remaining() {
|
||||
return buffer.readableBytes();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int position() {
|
||||
return buffer.readerIndex();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasRemaining() {
|
||||
return buffer.readableBytes() > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer duplicate() {
|
||||
return new NettyReadable(buffer.duplicate());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer byteBuffer() {
|
||||
return buffer.nioBuffer(0, buffer.writerIndex());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String readUTF8() {
|
||||
return buffer.toString(Charset_UTF8);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/**
|
||||
* 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.protocol.amqp.util;
|
||||
|
||||
import org.apache.qpid.proton.codec.AMQPDefinedTypes;
|
||||
import org.apache.qpid.proton.codec.DecoderImpl;
|
||||
import org.apache.qpid.proton.codec.EncoderImpl;
|
||||
|
||||
/** This can go away if Proton provides this feature. */
|
||||
public class TLSEncode {
|
||||
|
||||
// For now Proton requires that we create a decoder to create an encoder
|
||||
private static class EncoderDecoderPair {
|
||||
DecoderImpl decoder = new DecoderImpl();
|
||||
EncoderImpl encoder = new EncoderImpl(decoder);
|
||||
{
|
||||
AMQPDefinedTypes.registerAllTypes(decoder, encoder);
|
||||
}
|
||||
}
|
||||
|
||||
private static final ThreadLocal<EncoderDecoderPair> tlsCodec = new ThreadLocal<EncoderDecoderPair>() {
|
||||
@Override
|
||||
protected EncoderDecoderPair initialValue() {
|
||||
return new EncoderDecoderPair();
|
||||
}
|
||||
};
|
||||
|
||||
public static EncoderImpl getEncoder() {
|
||||
return tlsCodec.get().encoder;
|
||||
}
|
||||
|
||||
public static DecoderImpl getDecoder() {
|
||||
return tlsCodec.get().decoder;
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -16,44 +16,28 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||
|
||||
import static org.apache.activemq.artemis.api.core.Message.BYTES_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.MAP_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.STREAM_TYPE;
|
||||
import static org.apache.activemq.artemis.api.core.Message.TEXT_TYPE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.wrapMessage;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.journal.EncodingSupport;
|
||||
import org.apache.activemq.artemis.core.server.ServerMessage;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSBytesMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMapMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSStreamMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSTextMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.message.EncodedMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
import org.apache.activemq.artemis.utils.SimpleIDGenerator;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
|
||||
import org.apache.qpid.proton.amqp.messaging.AmqpValue;
|
||||
import org.apache.qpid.proton.amqp.messaging.ApplicationProperties;
|
||||
import org.apache.qpid.proton.amqp.messaging.Data;
|
||||
import org.apache.qpid.proton.message.Message;
|
||||
import org.apache.qpid.proton.message.ProtonJMessage;
|
||||
import org.apache.qpid.proton.message.impl.MessageImpl;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
|
||||
public class TestConversions extends Assert {
|
||||
|
||||
|
@ -72,18 +56,12 @@ public class TestConversions extends Assert {
|
|||
|
||||
message.setBody(new AmqpValue(new Boolean(true)));
|
||||
|
||||
EncodedMessage encodedMessage = encodeMessage(message);
|
||||
AMQPMessage encodedMessage = new AMQPMessage(message);
|
||||
|
||||
ProtonMessageConverter converter = new ProtonMessageConverter(new SimpleIDGenerator(0));
|
||||
ServerMessage serverMessage = converter.inbound(encodedMessage);
|
||||
ICoreMessage serverMessage = encodedMessage.toCore();
|
||||
|
||||
verifyProperties(new ServerJMSMessage(serverMessage, 0));
|
||||
verifyProperties(ServerJMSMessage.wrapCoreMessage(serverMessage));
|
||||
|
||||
EncodedMessage encoded = (EncodedMessage) converter.outbound(serverMessage, 0);
|
||||
Message amqpMessage = encoded.decode();
|
||||
|
||||
AmqpValue value = (AmqpValue) amqpMessage.getBody();
|
||||
assertEquals(value.getValue(), true);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -101,12 +79,11 @@ public class TestConversions extends Assert {
|
|||
|
||||
message.setBody(new Data(new Binary(bodyBytes)));
|
||||
|
||||
EncodedMessage encodedMessage = encodeMessage(message);
|
||||
AMQPMessage encodedMessage = new AMQPMessage(message);
|
||||
|
||||
ProtonMessageConverter converter = new ProtonMessageConverter(new SimpleIDGenerator(0));
|
||||
ServerMessage serverMessage = converter.inbound(encodedMessage);
|
||||
ICoreMessage serverMessage = encodedMessage.toCore();
|
||||
|
||||
ServerJMSBytesMessage bytesMessage = (ServerJMSBytesMessage) wrapMessage(BYTES_TYPE, serverMessage, 0);
|
||||
ServerJMSBytesMessage bytesMessage = (ServerJMSBytesMessage) ServerJMSMessage.wrapCoreMessage(serverMessage);
|
||||
|
||||
verifyProperties(bytesMessage);
|
||||
|
||||
|
@ -118,9 +95,6 @@ public class TestConversions extends Assert {
|
|||
|
||||
Assert.assertArrayEquals(bodyBytes, newBodyBytes);
|
||||
|
||||
Object obj = converter.outbound(serverMessage, 0);
|
||||
|
||||
System.out.println("output = " + obj);
|
||||
}
|
||||
|
||||
private void verifyProperties(javax.jms.Message message) throws Exception {
|
||||
|
@ -151,12 +125,12 @@ public class TestConversions extends Assert {
|
|||
|
||||
message.setBody(new AmqpValue(mapValues));
|
||||
|
||||
EncodedMessage encodedMessage = encodeMessage(message);
|
||||
AMQPMessage encodedMessage = new AMQPMessage(message);
|
||||
|
||||
ProtonMessageConverter converter = new ProtonMessageConverter(new SimpleIDGenerator(0));
|
||||
ServerMessage serverMessage = converter.inbound(encodedMessage);
|
||||
ICoreMessage serverMessage = encodedMessage.toCore();
|
||||
serverMessage.getReadOnlyBodyBuffer();
|
||||
|
||||
ServerJMSMapMessage mapMessage = (ServerJMSMapMessage) wrapMessage(MAP_TYPE, serverMessage, 0);
|
||||
ServerJMSMapMessage mapMessage = (ServerJMSMapMessage) ServerJMSMessage.wrapCoreMessage(serverMessage);
|
||||
mapMessage.decode();
|
||||
|
||||
verifyProperties(mapMessage);
|
||||
|
@ -164,15 +138,8 @@ public class TestConversions extends Assert {
|
|||
Assert.assertEquals(1, mapMessage.getInt("someint"));
|
||||
Assert.assertEquals("value", mapMessage.getString("somestr"));
|
||||
|
||||
EncodedMessage encoded = (EncodedMessage) converter.outbound(serverMessage, 0);
|
||||
Message amqpMessage = encoded.decode();
|
||||
|
||||
AmqpValue value = (AmqpValue) amqpMessage.getBody();
|
||||
Map<?, ?> mapoutput = (Map<?, ?>) value.getValue();
|
||||
|
||||
assertEquals(Integer.valueOf(1), mapoutput.get("someint"));
|
||||
|
||||
System.out.println("output = " + amqpMessage);
|
||||
AMQPMessage newAMQP = CoreAmqpConverter.fromCore(mapMessage.getInnerMessage());
|
||||
System.out.println(newAMQP.getProtonMessage().getBody());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -188,14 +155,11 @@ public class TestConversions extends Assert {
|
|||
|
||||
message.setBody(new AmqpSequence(objects));
|
||||
|
||||
EncodedMessage encodedMessage = encodeMessage(message);
|
||||
AMQPMessage encodedMessage = new AMQPMessage(message);
|
||||
|
||||
ProtonMessageConverter converter = new ProtonMessageConverter(new SimpleIDGenerator(0));
|
||||
ServerMessage serverMessage = converter.inbound(encodedMessage);
|
||||
ICoreMessage serverMessage = encodedMessage.toCore();
|
||||
|
||||
simulatePersistence(serverMessage);
|
||||
|
||||
ServerJMSStreamMessage streamMessage = (ServerJMSStreamMessage) wrapMessage(STREAM_TYPE, serverMessage, 0);
|
||||
ServerJMSStreamMessage streamMessage = (ServerJMSStreamMessage) ServerJMSMessage.wrapCoreMessage(serverMessage);
|
||||
|
||||
verifyProperties(streamMessage);
|
||||
|
||||
|
@ -203,13 +167,6 @@ public class TestConversions extends Assert {
|
|||
|
||||
assertEquals(10, streamMessage.readInt());
|
||||
assertEquals("10", streamMessage.readString());
|
||||
|
||||
EncodedMessage encoded = (EncodedMessage) converter.outbound(serverMessage, 0);
|
||||
Message amqpMessage = encoded.decode();
|
||||
|
||||
List<?> list = ((AmqpSequence) amqpMessage.getBody()).getValue();
|
||||
Assert.assertEquals(Integer.valueOf(10), list.get(0));
|
||||
Assert.assertEquals("10", list.get(1));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -222,553 +179,17 @@ public class TestConversions extends Assert {
|
|||
String text = "someText";
|
||||
message.setBody(new AmqpValue(text));
|
||||
|
||||
EncodedMessage encodedMessage = encodeMessage(message);
|
||||
AMQPMessage encodedMessage = new AMQPMessage(message);
|
||||
|
||||
ProtonMessageConverter converter = new ProtonMessageConverter(new SimpleIDGenerator(0));
|
||||
ServerMessage serverMessage = converter.inbound(encodedMessage);
|
||||
ICoreMessage serverMessage = encodedMessage.toCore();
|
||||
|
||||
simulatePersistence(serverMessage);
|
||||
|
||||
ServerJMSTextMessage textMessage = (ServerJMSTextMessage) wrapMessage(TEXT_TYPE, serverMessage, 0);
|
||||
ServerJMSTextMessage textMessage = (ServerJMSTextMessage) ServerJMSMessage.wrapCoreMessage(serverMessage);
|
||||
textMessage.decode();
|
||||
|
||||
verifyProperties(textMessage);
|
||||
|
||||
Assert.assertEquals(text, textMessage.getText());
|
||||
|
||||
EncodedMessage encoded = (EncodedMessage) converter.outbound(serverMessage, 0);
|
||||
Message amqpMessage = encoded.decode();
|
||||
|
||||
AmqpValue value = (AmqpValue) amqpMessage.getBody();
|
||||
String textValue = (String) value.getValue();
|
||||
|
||||
Assert.assertEquals(text, textValue);
|
||||
|
||||
System.out.println("output = " + amqpMessage);
|
||||
}
|
||||
|
||||
private void simulatePersistence(ServerMessage serverMessage) {
|
||||
serverMessage.setAddress(new SimpleString("SomeAddress"));
|
||||
// This is just to simulate what would happen during the persistence of the message
|
||||
// We need to still be able to recover the message when we read it back
|
||||
((EncodingSupport) serverMessage).encode(new EmptyBuffer());
|
||||
}
|
||||
|
||||
private ProtonJMessage reEncodeMsg(Object obj) {
|
||||
ProtonJMessage objOut = (ProtonJMessage) obj;
|
||||
|
||||
ByteBuf nettyBuffer = PooledByteBufAllocator.DEFAULT.heapBuffer(1024);
|
||||
|
||||
objOut.encode(new NettyWritable(nettyBuffer));
|
||||
return objOut;
|
||||
}
|
||||
|
||||
private EncodedMessage encodeMessage(MessageImpl message) {
|
||||
ByteBuf buf = PooledByteBufAllocator.DEFAULT.heapBuffer(1024 * 1024);
|
||||
message.encode(new NettyWritable(buf));
|
||||
byte[] bytesConvert = new byte[buf.writerIndex()];
|
||||
buf.readBytes(bytesConvert);
|
||||
return new EncodedMessage(0, bytesConvert, 0, bytesConvert.length);
|
||||
}
|
||||
|
||||
class EmptyBuffer implements ActiveMQBuffer {
|
||||
|
||||
@Override
|
||||
public ByteBuf byteBuf() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int capacity() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readerIndex() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readerIndex(int readerIndex) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int writerIndex() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writerIndex(int writerIndex) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setIndex(int readerIndex, int writerIndex) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readableBytes() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int writableBytes() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean readable() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean writable() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markReaderIndex() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void resetReaderIndex() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markWriterIndex() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void resetWriterIndex() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void discardReadBytes() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getByte(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getUnsignedByte(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getShort(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUnsignedShort(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getUnsignedInt(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getBytes(int index, ActiveMQBuffer dst) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getBytes(int index, ActiveMQBuffer dst, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getBytes(int index, ActiveMQBuffer dst, int dstIndex, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getBytes(int index, byte[] dst) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getBytes(int index, byte[] dst, int dstIndex, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getBytes(int index, ByteBuffer dst) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public char getChar(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int index) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setByte(int index, byte value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setShort(int index, short value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setInt(int index, int value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setLong(int index, long value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBytes(int index, ActiveMQBuffer src) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBytes(int index, ActiveMQBuffer src, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBytes(int index, ActiveMQBuffer src, int srcIndex, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBytes(int index, byte[] src) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBytes(int index, byte[] src, int srcIndex, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setBytes(int index, ByteBuffer src) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setChar(int index, char value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setFloat(int index, float value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDouble(int index, double value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte readByte() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readUnsignedByte() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short readShort() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readUnsignedShort() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readInt() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long readUnsignedInt() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long readLong() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public char readChar() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float readFloat() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double readDouble() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean readBoolean() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimpleString readNullableSimpleString() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String readNullableString() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimpleString readSimpleString() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String readString() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String readUTF() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActiveMQBuffer readBytes(int length) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActiveMQBuffer readSlice(int length) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readBytes(ActiveMQBuffer dst) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readBytes(ActiveMQBuffer dst, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readBytes(ActiveMQBuffer dst, int dstIndex, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readBytes(byte[] dst) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readBytes(byte[] dst, int dstIndex, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readBytes(ByteBuffer dst) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int skipBytes(int length) {
|
||||
return length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeByte(byte value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeShort(short value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeInt(int value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeLong(long value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeChar(char chr) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeFloat(float value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeDouble(double value) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBoolean(boolean val) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeNullableSimpleString(SimpleString val) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeNullableString(String val) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeSimpleString(SimpleString val) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeString(String val) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeUTF(String utf) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(ActiveMQBuffer src, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(ActiveMQBuffer src, int srcIndex, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(byte[] src) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(byte[] src, int srcIndex, int length) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(ByteBuffer src) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFully(byte[] b) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFully(byte[] b, int off, int len) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public String readLine() throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActiveMQBuffer copy() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActiveMQBuffer copy(int index, int length) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActiveMQBuffer slice() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActiveMQBuffer slice(int index, int length) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActiveMQBuffer duplicate() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer toByteBuffer() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer toByteBuffer(int index, int length) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void release() {
|
||||
//no-op
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,15 +16,17 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPContentTypeSupport;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPInvalidContentTypeException;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
||||
public class AMQPContentTypeSupportTest {
|
||||
|
||||
@Test(expected = ActiveMQAMQPInvalidContentTypeException.class)
|
||||
|
@ -216,7 +218,7 @@ public class AMQPContentTypeSupportTest {
|
|||
@Test
|
||||
public void testParseContentTypeWithApplicationJavaSerialized() throws Exception {
|
||||
// Expect null as this is not a textual type
|
||||
doParseContentTypeTestImpl(AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE, null);
|
||||
doParseContentTypeTestImpl(AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE.toString(), null);
|
||||
}
|
||||
|
||||
private void doParseContentTypeTestImpl(String contentType, Charset expected) throws ActiveMQAMQPInvalidContentTypeException {
|
||||
|
|
|
@ -20,19 +20,20 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageIdHelper;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPException;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.amqp.UnsignedLong;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class AMQPMessageIdHelperTest {
|
||||
|
||||
private AMQPMessageIdHelper messageIdHelper;
|
||||
|
|
|
@ -16,20 +16,21 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport;
|
||||
import org.apache.qpid.proton.Proton;
|
||||
import org.apache.qpid.proton.amqp.Symbol;
|
||||
import org.apache.qpid.proton.amqp.messaging.MessageAnnotations;
|
||||
import org.apache.qpid.proton.message.Message;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class AMQPMessageSupportTest {
|
||||
|
||||
// ---------- getSymbol ---------------------------------------------------//
|
||||
|
|
|
@ -16,36 +16,29 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import javax.jms.Destination;
|
||||
import javax.jms.MapMessage;
|
||||
import javax.jms.Queue;
|
||||
import javax.jms.TemporaryQueue;
|
||||
import javax.jms.TemporaryTopic;
|
||||
import javax.jms.TextMessage;
|
||||
import javax.jms.Topic;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQMessage;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSBytesMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMapMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSObjectMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSStreamMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSTextMessage;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.activemq.artemis.utils.SimpleIDGenerator;
|
||||
import org.apache.qpid.proton.Proton;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.amqp.Symbol;
|
||||
|
@ -57,13 +50,15 @@ import org.apache.qpid.proton.message.Message;
|
|||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class JMSMappingInboundTransformerTest {
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
private IDGenerator idGenerator;
|
||||
public class JMSMappingInboundTransformerTest {
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
this.idGenerator = new SimpleIDGenerator(0);
|
||||
}
|
||||
|
||||
// ----- Null Body Section ------------------------------------------------//
|
||||
|
@ -77,13 +72,14 @@ public class JMSMappingInboundTransformerTest {
|
|||
*/
|
||||
@Test
|
||||
public void testCreateBytesMessageFromNoBodySectionAndContentType() throws Exception {
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
|
||||
Message message = Message.Factory.create();
|
||||
message.setContentType(AMQPMessageSupport.OCTET_STREAM_CONTENT_TYPE);
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
AMQPMessage messageEncode = new AMQPMessage(message);
|
||||
|
||||
ICoreMessage coreMessage = messageEncode.toCore();
|
||||
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(coreMessage);
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSBytesMessage.class, jmsMessage.getClass());
|
||||
|
@ -98,74 +94,25 @@ public class JMSMappingInboundTransformerTest {
|
|||
*/
|
||||
@Test
|
||||
public void testCreateBytesMessageFromNoBodySectionAndNoContentType() throws Exception {
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
|
||||
Message message = Message.Factory.create();
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSBytesMessage.class, jmsMessage.getClass());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that a message with no body section, but with the content type set to
|
||||
* {@value AMQPMessageSupport#SERIALIZED_JAVA_OBJECT_CONTENT_TYPE} results in an
|
||||
* ObjectMessage when not otherwise annotated to indicate the type of JMS message it is.
|
||||
*
|
||||
* @throws Exception
|
||||
* if an error occurs during the test.
|
||||
*/
|
||||
@Test
|
||||
public void testCreateObjectMessageFromNoBodySectionAndContentType() throws Exception {
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
|
||||
Message message = Message.Factory.create();
|
||||
message.setContentType(AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSObjectMessage.class, jmsMessage.getClass());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateTextMessageFromNoBodySectionAndContentType() throws Exception {
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
|
||||
Message message = Message.Factory.create();
|
||||
message.setContentType("text/plain");
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSTextMessage.class, jmsMessage.getClass());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that a message with no body section, and with the content type set to an unknown
|
||||
* value results in a plain Message when not otherwise annotated to indicate the type of JMS
|
||||
* message it is.
|
||||
*
|
||||
* @throws Exception
|
||||
* if an error occurs during the test.
|
||||
*/
|
||||
public void testCreateGenericMessageFromNoBodySectionAndUnknownContentType() throws Exception {
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
|
||||
Message message = Message.Factory.create();
|
||||
message.setContentType("unknown-content-type");
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ActiveMQMessage.class, jmsMessage.getClass());
|
||||
}
|
||||
|
||||
// ----- Data Body Section ------------------------------------------------//
|
||||
|
||||
/**
|
||||
|
@ -183,10 +130,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
message.setBody(new Data(binary));
|
||||
message.setContentType(AMQPMessageSupport.OCTET_STREAM_CONTENT_TYPE);
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSBytesMessage.class, jmsMessage.getClass());
|
||||
|
@ -206,10 +150,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
message.setBody(new Data(binary));
|
||||
message.setContentType("unknown-content-type");
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSBytesMessage.class, jmsMessage.getClass());
|
||||
|
@ -230,10 +171,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
|
||||
assertNull(message.getContentType());
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSBytesMessage.class, jmsMessage.getClass());
|
||||
|
@ -252,12 +190,9 @@ public class JMSMappingInboundTransformerTest {
|
|||
Message message = Proton.message();
|
||||
Binary binary = new Binary(new byte[0]);
|
||||
message.setBody(new Data(binary));
|
||||
message.setContentType(AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
|
||||
message.setContentType(AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE.toString());
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSObjectMessage.class, jmsMessage.getClass());
|
||||
|
@ -357,10 +292,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
message.setBody(new Data(binary));
|
||||
message.setContentType(contentType);
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
if (StandardCharsets.UTF_8.equals(expectedCharset)) {
|
||||
|
@ -384,10 +316,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
Message message = Proton.message();
|
||||
message.setBody(new AmqpValue("content"));
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSTextMessage.class, jmsMessage.getClass());
|
||||
|
@ -405,10 +334,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
Message message = Proton.message();
|
||||
message.setBody(new AmqpValue(null));
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSTextMessage.class, jmsMessage.getClass());
|
||||
|
@ -424,14 +350,11 @@ public class JMSMappingInboundTransformerTest {
|
|||
*/
|
||||
@Test
|
||||
public void testCreateObjectMessageFromAmqpValueWithBinaryAndContentType() throws Exception {
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
|
||||
Message message = Message.Factory.create();
|
||||
message.setBody(new AmqpValue(new Binary(new byte[0])));
|
||||
message.setContentType(AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE);
|
||||
message.setContentType(AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE.toString());
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSObjectMessage.class, jmsMessage.getClass());
|
||||
|
@ -450,49 +373,12 @@ public class JMSMappingInboundTransformerTest {
|
|||
Map<String, String> map = new HashMap<>();
|
||||
message.setBody(new AmqpValue(map));
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSMapMessage.class, jmsMessage.getClass());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that an amqp-value body containing a map that has an AMQP Binary as one of the
|
||||
* entries encoded into the Map results in an MapMessage where a byte array can be read from
|
||||
* the entry.
|
||||
*
|
||||
* @throws Exception
|
||||
* if an error occurs during the test.
|
||||
*/
|
||||
@Test
|
||||
public void testCreateAmqpMapMessageFromAmqpValueWithMapContainingBinaryEntry() throws Exception {
|
||||
final String ENTRY_NAME = "bytesEntry";
|
||||
|
||||
Message message = Proton.message();
|
||||
Map<String, Object> map = new HashMap<>();
|
||||
|
||||
byte[] inputBytes = new byte[] {1, 2, 3, 4, 5};
|
||||
map.put(ENTRY_NAME, new Binary(inputBytes));
|
||||
|
||||
message.setBody(new AmqpValue(map));
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSMapMessage.class, jmsMessage.getClass());
|
||||
|
||||
MapMessage mapMessage = (MapMessage) jmsMessage;
|
||||
byte[] outputBytes = mapMessage.getBytes(ENTRY_NAME);
|
||||
assertNotNull(outputBytes);
|
||||
assertTrue(Arrays.equals(inputBytes, outputBytes));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that an amqp-value body containing a list results in an StreamMessage when not
|
||||
* otherwise annotated to indicate the type of JMS message it is.
|
||||
|
@ -506,10 +392,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
List<String> list = new ArrayList<>();
|
||||
message.setBody(new AmqpValue(list));
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSStreamMessage.class, jmsMessage.getClass());
|
||||
|
@ -528,10 +411,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
List<String> list = new ArrayList<>();
|
||||
message.setBody(new AmqpSequence(list));
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSStreamMessage.class, jmsMessage.getClass());
|
||||
|
@ -550,10 +430,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
Binary binary = new Binary(new byte[0]);
|
||||
message.setBody(new AmqpValue(binary));
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSBytesMessage.class, jmsMessage.getClass());
|
||||
|
@ -572,11 +449,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
Message message = Proton.message();
|
||||
message.setBody(new AmqpValue(UUID.randomUUID()));
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSBytesMessage.class, jmsMessage.getClass());
|
||||
|
@ -588,10 +461,8 @@ public class JMSMappingInboundTransformerTest {
|
|||
Message message = Message.Factory.create();
|
||||
message.setBody(new AmqpValue(contentString));
|
||||
|
||||
EncodedMessage em = encodeMessage(message);
|
||||
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
ServerJMSTextMessage jmsMessage = (ServerJMSTextMessage)ServerJMSMessage.wrapCoreMessage(new AMQPMessage(message).toCore());
|
||||
jmsMessage.decode();
|
||||
|
||||
assertTrue("Expected TextMessage", jmsMessage instanceof TextMessage);
|
||||
assertEquals("Unexpected message class type", ServerJMSTextMessage.class, jmsMessage.getClass());
|
||||
|
@ -631,7 +502,6 @@ public class JMSMappingInboundTransformerTest {
|
|||
|
||||
private void doTransformWithToTypeDestinationTypeAnnotationTestImpl(Object toTypeAnnotationValue, Class<? extends Destination> expectedClass)
|
||||
throws Exception {
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
|
||||
String toAddress = "toAddress";
|
||||
Message amqp = Message.Factory.create();
|
||||
|
@ -644,9 +514,7 @@ public class JMSMappingInboundTransformerTest {
|
|||
amqp.setMessageAnnotations(ma);
|
||||
}
|
||||
|
||||
EncodedMessage em = encodeMessage(amqp);
|
||||
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(amqp).toCore());
|
||||
assertTrue("Expected TextMessage", jmsMessage instanceof TextMessage);
|
||||
}
|
||||
|
||||
|
@ -679,7 +547,6 @@ public class JMSMappingInboundTransformerTest {
|
|||
|
||||
private void doTransformWithReplyToTypeDestinationTypeAnnotationTestImpl(Object replyToTypeAnnotationValue, Class<? extends Destination> expectedClass)
|
||||
throws Exception {
|
||||
JMSMappingInboundTransformer transformer = new JMSMappingInboundTransformer(idGenerator);
|
||||
|
||||
String replyToAddress = "replyToAddress";
|
||||
Message amqp = Message.Factory.create();
|
||||
|
@ -692,27 +559,8 @@ public class JMSMappingInboundTransformerTest {
|
|||
amqp.setMessageAnnotations(ma);
|
||||
}
|
||||
|
||||
EncodedMessage em = encodeMessage(amqp);
|
||||
|
||||
javax.jms.Message jmsMessage = transformer.transform(em);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(new AMQPMessage(amqp).toCore());
|
||||
assertTrue("Expected TextMessage", jmsMessage instanceof TextMessage);
|
||||
}
|
||||
|
||||
// ----- Utility Methods --------------------------------------------------//
|
||||
|
||||
private EncodedMessage encodeMessage(Message message) {
|
||||
byte[] encodeBuffer = new byte[1024 * 8];
|
||||
int encodedSize;
|
||||
while (true) {
|
||||
try {
|
||||
encodedSize = message.encode(encodeBuffer, 0, encodeBuffer.length);
|
||||
break;
|
||||
} catch (java.nio.BufferOverflowException e) {
|
||||
encodeBuffer = new byte[encodeBuffer.length * 2];
|
||||
}
|
||||
}
|
||||
|
||||
long messageFormat = 0;
|
||||
return new EncodedMessage(messageFormat, encodeBuffer, 0, encodedSize);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,33 +16,20 @@
|
|||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter.message;
|
||||
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_DATA;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_NULL;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_SEQUENCE;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_UNKNOWN;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.AMQP_VALUE_BINARY;
|
||||
import static org.apache.activemq.artemis.protocol.amqp.converter.message.AMQPMessageSupport.JMS_AMQP_ORIGINAL_ENCODING;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import javax.jms.JMSException;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.ObjectInputStream;
|
||||
import java.io.ObjectOutputStream;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import javax.jms.JMSException;
|
||||
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ResetLimitWrappedActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerMessageImpl;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPConverter;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerDestination;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSBytesMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMapMessage;
|
||||
|
@ -50,9 +37,6 @@ import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
|||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSObjectMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSStreamMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSTextMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
import org.apache.activemq.artemis.utils.IDGenerator;
|
||||
import org.apache.activemq.artemis.utils.SimpleIDGenerator;
|
||||
import org.apache.qpid.proton.amqp.Binary;
|
||||
import org.apache.qpid.proton.amqp.Symbol;
|
||||
import org.apache.qpid.proton.amqp.messaging.AmqpSequence;
|
||||
|
@ -64,16 +48,18 @@ import org.junit.Before;
|
|||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class JMSMappingOutboundTransformerTest {
|
||||
|
||||
private final UUID TEST_OBJECT_VALUE = UUID.fromString("fee14b62-09e0-4ac6-a4c3-4206c630d844");
|
||||
private final String TEST_ADDRESS = "queue://testAddress";
|
||||
|
||||
private IDGenerator idGenerator;
|
||||
private JMSMappingOutboundTransformer transformer;
|
||||
|
||||
public static final byte QUEUE_TYPE = 0x00;
|
||||
public static final byte TOPIC_TYPE = 0x01;
|
||||
|
@ -82,80 +68,10 @@ public class JMSMappingOutboundTransformerTest {
|
|||
|
||||
@Before
|
||||
public void setUp() {
|
||||
idGenerator = new SimpleIDGenerator(0);
|
||||
transformer = new JMSMappingOutboundTransformer(idGenerator);
|
||||
}
|
||||
|
||||
// ----- no-body Message type tests ---------------------------------------//
|
||||
|
||||
@Test
|
||||
public void testConvertMessageToAmqpMessageWithNoBody() throws Exception {
|
||||
ServerJMSMessage outbound = createMessage();
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
|
||||
assertNull(amqp.getBody());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertTextMessageToAmqpMessageWithNoBodyOriginalEncodingWasNull() throws Exception {
|
||||
ServerJMSTextMessage outbound = createTextMessage();
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_NULL);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
|
||||
assertNull(amqp.getBody());
|
||||
}
|
||||
|
||||
// ----- BytesMessage type tests ---------------------------------------//
|
||||
|
||||
@Test
|
||||
public void testConvertEmptyBytesMessageToAmqpMessageWithDataBody() throws Exception {
|
||||
ServerJMSBytesMessage outbound = createBytesMessage();
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
assertTrue(((Data) amqp.getBody()).getValue() instanceof Binary);
|
||||
assertEquals(0, ((Data) amqp.getBody()).getValue().getLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertUncompressedBytesMessageToAmqpMessageWithDataBody() throws Exception {
|
||||
byte[] expectedPayload = new byte[] {8, 16, 24, 32};
|
||||
ServerJMSBytesMessage outbound = createBytesMessage();
|
||||
outbound.writeBytes(expectedPayload);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
assertTrue(((Data) amqp.getBody()).getValue() instanceof Binary);
|
||||
assertEquals(4, ((Data) amqp.getBody()).getValue().getLength());
|
||||
|
||||
Binary amqpData = ((Data) amqp.getBody()).getValue();
|
||||
Binary inputData = new Binary(expectedPayload);
|
||||
|
||||
assertTrue(inputData.equals(amqpData));
|
||||
}
|
||||
|
||||
@Ignore("Compressed message body support not yet implemented.")
|
||||
@Test
|
||||
public void testConvertCompressedBytesMessageToAmqpMessageWithDataBody() throws Exception {
|
||||
|
@ -164,10 +80,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
outbound.writeBytes(expectedPayload);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
|
@ -183,13 +96,9 @@ public class JMSMappingOutboundTransformerTest {
|
|||
@Test
|
||||
public void testConvertEmptyBytesMessageToAmqpMessageWithAmqpValueBody() throws Exception {
|
||||
ServerJMSBytesMessage outbound = createBytesMessage();
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
@ -201,14 +110,10 @@ public class JMSMappingOutboundTransformerTest {
|
|||
public void testConvertUncompressedBytesMessageToAmqpMessageWithAmqpValueBody() throws Exception {
|
||||
byte[] expectedPayload = new byte[] {8, 16, 24, 32};
|
||||
ServerJMSBytesMessage outbound = createBytesMessage();
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
outbound.writeBytes(expectedPayload);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
@ -226,14 +131,10 @@ public class JMSMappingOutboundTransformerTest {
|
|||
public void testConvertCompressedBytesMessageToAmqpMessageWithAmqpValueBody() throws Exception {
|
||||
byte[] expectedPayload = new byte[] {8, 16, 24, 32};
|
||||
ServerJMSBytesMessage outbound = createBytesMessage(true);
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
outbound.writeBytes(expectedPayload);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
@ -253,10 +154,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
ServerJMSMapMessage outbound = createMapMessage();
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
@ -271,10 +169,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
outbound.setBytes("bytes", byteArray);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
@ -296,10 +191,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
outbound.setBoolean("property-3", true);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
@ -320,10 +212,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
outbound.setBoolean("property-3", true);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
@ -336,33 +225,12 @@ public class JMSMappingOutboundTransformerTest {
|
|||
assertTrue("string".equals(amqpMap.get("property-1")));
|
||||
}
|
||||
|
||||
// ----- StreamMessage type tests -----------------------------------------//
|
||||
|
||||
@Test
|
||||
public void testConvertStreamMessageToAmqpMessageWithAmqpValueBody() throws Exception {
|
||||
ServerJMSStreamMessage outbound = createStreamMessage();
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
assertTrue(((AmqpValue) amqp.getBody()).getValue() instanceof List);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertStreamMessageToAmqpMessageWithAmqpSequencey() throws Exception {
|
||||
ServerJMSStreamMessage outbound = createStreamMessage();
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_SEQUENCE);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpSequence);
|
||||
|
@ -376,17 +244,15 @@ public class JMSMappingOutboundTransformerTest {
|
|||
outbound.writeString("test");
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
assertTrue(((AmqpValue) amqp.getBody()).getValue() instanceof List);
|
||||
assertTrue(amqp.getBody() instanceof AmqpSequence);
|
||||
|
||||
AmqpSequence list = (AmqpSequence)amqp.getBody();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
List<Object> amqpList = (List<Object>) ((AmqpValue) amqp.getBody()).getValue();
|
||||
List<Object> amqpList = list.getValue();
|
||||
|
||||
assertEquals(2, amqpList.size());
|
||||
}
|
||||
|
@ -394,15 +260,11 @@ public class JMSMappingOutboundTransformerTest {
|
|||
@Test
|
||||
public void testConvertCompressedStreamMessageToAmqpMessageWithAmqpSequencey() throws Exception {
|
||||
ServerJMSStreamMessage outbound = createStreamMessage(true);
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_SEQUENCE);
|
||||
outbound.writeBoolean(false);
|
||||
outbound.writeString("test");
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpSequence);
|
||||
|
@ -421,10 +283,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
ServerJMSObjectMessage outbound = createObjectMessage();
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
|
@ -434,45 +293,20 @@ public class JMSMappingOutboundTransformerTest {
|
|||
@Test
|
||||
public void testConvertEmptyObjectMessageToAmqpMessageUnknownEncodingGetsDataSection() throws Exception {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage();
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_UNKNOWN);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
assertEquals(5, ((Data) amqp.getBody()).getValue().getLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertEmptyObjectMessageToAmqpMessageWithAmqpValueBody() throws Exception {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage();
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
assertTrue(((AmqpValue) amqp.getBody()).getValue() instanceof Binary);
|
||||
assertEquals(5, ((Binary) ((AmqpValue) amqp.getBody()).getValue()).getLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertObjectMessageToAmqpMessageWithDataBody() throws Exception {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage(TEST_OBJECT_VALUE);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
|
@ -486,13 +320,9 @@ public class JMSMappingOutboundTransformerTest {
|
|||
@Test
|
||||
public void testConvertObjectMessageToAmqpMessageUnknownEncodingGetsDataSection() throws Exception {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage(TEST_OBJECT_VALUE);
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_UNKNOWN);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
|
@ -503,36 +333,12 @@ public class JMSMappingOutboundTransformerTest {
|
|||
assertTrue(value instanceof UUID);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertObjectMessageToAmqpMessageWithAmqpValueBody() throws Exception {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage(TEST_OBJECT_VALUE);
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
assertTrue(((AmqpValue) amqp.getBody()).getValue() instanceof Binary);
|
||||
assertFalse(0 == ((Binary) ((AmqpValue) amqp.getBody()).getValue()).getLength());
|
||||
|
||||
Object value = deserialize(((Binary) ((AmqpValue) amqp.getBody()).getValue()).getArray());
|
||||
assertNotNull(value);
|
||||
assertTrue(value instanceof UUID);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertCompressedObjectMessageToAmqpMessageWithDataBody() throws Exception {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage(TEST_OBJECT_VALUE, true);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
|
@ -546,13 +352,9 @@ public class JMSMappingOutboundTransformerTest {
|
|||
@Test
|
||||
public void testConvertCompressedObjectMessageToAmqpMessageUnknownEncodingGetsDataSection() throws Exception {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage(TEST_OBJECT_VALUE, true);
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_UNKNOWN);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
|
@ -566,20 +368,16 @@ public class JMSMappingOutboundTransformerTest {
|
|||
@Test
|
||||
public void testConvertCompressedObjectMessageToAmqpMessageWithAmqpValueBody() throws Exception {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage(TEST_OBJECT_VALUE, true);
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
assertTrue(((AmqpValue) amqp.getBody()).getValue() instanceof Binary);
|
||||
assertFalse(0 == ((Binary) ((AmqpValue) amqp.getBody()).getValue()).getLength());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
assertTrue(((Data) amqp.getBody()).getValue() instanceof Binary);
|
||||
assertFalse(0 == ((Binary) ((Data) amqp.getBody()).getValue()).getLength());
|
||||
|
||||
Object value = deserialize(((Binary) ((AmqpValue) amqp.getBody()).getValue()).getArray());
|
||||
Object value = deserialize((((Data) amqp.getBody()).getValue()).getArray());
|
||||
assertNotNull(value);
|
||||
assertTrue(value instanceof UUID);
|
||||
}
|
||||
|
@ -591,68 +389,20 @@ public class JMSMappingOutboundTransformerTest {
|
|||
ServerJMSTextMessage outbound = createTextMessage();
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
assertNull(((AmqpValue) amqp.getBody()).getValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertTextMessageCreatesBodyUsingOriginalEncodingWithDataSection() throws Exception {
|
||||
String contentString = "myTextMessageContent";
|
||||
ServerJMSTextMessage outbound = createTextMessage(contentString);
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_DATA);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
assertTrue(((Data) amqp.getBody()).getValue() instanceof Binary);
|
||||
|
||||
Binary data = ((Data) amqp.getBody()).getValue();
|
||||
String contents = new String(data.getArray(), data.getArrayOffset(), data.getLength(), StandardCharsets.UTF_8);
|
||||
assertEquals(contentString, contents);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertTextMessageContentNotStoredCreatesBodyUsingOriginalEncodingWithDataSection() throws Exception {
|
||||
String contentString = "myTextMessageContent";
|
||||
ServerJMSTextMessage outbound = createTextMessage(contentString);
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_DATA);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
assertTrue(((Data) amqp.getBody()).getValue() instanceof Binary);
|
||||
|
||||
Binary data = ((Data) amqp.getBody()).getValue();
|
||||
String contents = new String(data.getArray(), data.getArrayOffset(), data.getLength(), StandardCharsets.UTF_8);
|
||||
assertEquals(contentString, contents);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertTextMessageCreatesAmqpValueStringBody() throws Exception {
|
||||
String contentString = "myTextMessageContent";
|
||||
ServerJMSTextMessage outbound = createTextMessage(contentString);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
@ -665,10 +415,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
ServerJMSTextMessage outbound = createTextMessage(contentString);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
@ -679,21 +426,16 @@ public class JMSMappingOutboundTransformerTest {
|
|||
public void testConvertCompressedTextMessageCreatesDataSectionBody() throws Exception {
|
||||
String contentString = "myTextMessageContent";
|
||||
ServerJMSTextMessage outbound = createTextMessage(contentString, true);
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_DATA);
|
||||
outbound.encode();
|
||||
|
||||
EncodedMessage encoded = transform(outbound);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage()).getProtonMessage();
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
assertTrue(((Data) amqp.getBody()).getValue() instanceof Binary);
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
|
||||
Binary data = ((Data) amqp.getBody()).getValue();
|
||||
String contents = new String(data.getArray(), data.getArrayOffset(), data.getLength(), StandardCharsets.UTF_8);
|
||||
assertEquals(contentString, contents);
|
||||
AmqpValue value = (AmqpValue)amqp.getBody();
|
||||
|
||||
assertEquals(contentString, value.getValue());
|
||||
}
|
||||
|
||||
// ----- Test JMSDestination Handling -------------------------------------//
|
||||
|
@ -731,15 +473,12 @@ public class JMSMappingOutboundTransformerTest {
|
|||
textMessage.setText("myTextMessageContent");
|
||||
textMessage.setJMSDestination(jmsDestination);
|
||||
|
||||
EncodedMessage encoded = transform(textMessage);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(textMessage.getInnerMessage()).getProtonMessage();
|
||||
|
||||
MessageAnnotations ma = amqp.getMessageAnnotations();
|
||||
Map<Symbol, Object> maMap = ma == null ? null : ma.getValue();
|
||||
if (maMap != null) {
|
||||
Object actualValue = maMap.get(JMSMappingOutboundTransformer.JMS_DEST_TYPE_MSG_ANNOTATION);
|
||||
Object actualValue = maMap.get(AMQPMessageSupport.JMS_DEST_TYPE_MSG_ANNOTATION);
|
||||
assertEquals("Unexpected annotation value", expectedAnnotationValue, actualValue);
|
||||
} else if (expectedAnnotationValue != null) {
|
||||
fail("Expected annotation value, but there were no annotations");
|
||||
|
@ -785,15 +524,12 @@ public class JMSMappingOutboundTransformerTest {
|
|||
textMessage.setText("myTextMessageContent");
|
||||
textMessage.setJMSReplyTo(jmsReplyTo);
|
||||
|
||||
EncodedMessage encoded = transform(textMessage);
|
||||
assertNotNull(encoded);
|
||||
|
||||
Message amqp = encoded.decode();
|
||||
Message amqp = AMQPConverter.getInstance().fromCore(textMessage.getInnerMessage()).getProtonMessage();
|
||||
|
||||
MessageAnnotations ma = amqp.getMessageAnnotations();
|
||||
Map<Symbol, Object> maMap = ma == null ? null : ma.getValue();
|
||||
if (maMap != null) {
|
||||
Object actualValue = maMap.get(JMSMappingOutboundTransformer.JMS_REPLY_TO_TYPE_MSG_ANNOTATION);
|
||||
Object actualValue = maMap.get(AMQPMessageSupport.JMS_REPLY_TO_TYPE_MSG_ANNOTATION);
|
||||
assertEquals("Unexpected annotation value", expectedAnnotationValue, actualValue);
|
||||
} else if (expectedAnnotationValue != null) {
|
||||
fail("Expected annotation value, but there were no annotations");
|
||||
|
@ -806,17 +542,6 @@ public class JMSMappingOutboundTransformerTest {
|
|||
|
||||
// ----- Utility Methods used for this Test -------------------------------//
|
||||
|
||||
public EncodedMessage transform(ServerJMSMessage message) throws Exception {
|
||||
// Useful for testing but not recommended for real life use.
|
||||
ByteBuf nettyBuffer = Unpooled.buffer(1024);
|
||||
NettyWritable buffer = new NettyWritable(nettyBuffer);
|
||||
|
||||
long messageFormat = transformer.transform(message, buffer);
|
||||
|
||||
EncodedMessage encoded = new EncodedMessage(messageFormat, nettyBuffer.array(), nettyBuffer.arrayOffset() + nettyBuffer.readerIndex(), nettyBuffer.readableBytes());
|
||||
|
||||
return encoded;
|
||||
}
|
||||
|
||||
private ServerDestination createDestination(byte destType) {
|
||||
ServerDestination destination = null;
|
||||
|
@ -841,7 +566,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
}
|
||||
|
||||
private ServerJMSMessage createMessage() {
|
||||
return new ServerJMSMessage(newMessage(org.apache.activemq.artemis.api.core.Message.DEFAULT_TYPE), 0);
|
||||
return new ServerJMSMessage(newMessage(org.apache.activemq.artemis.api.core.Message.DEFAULT_TYPE));
|
||||
}
|
||||
|
||||
private ServerJMSBytesMessage createBytesMessage() {
|
||||
|
@ -849,7 +574,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
}
|
||||
|
||||
private ServerJMSBytesMessage createBytesMessage(boolean compression) {
|
||||
ServerJMSBytesMessage message = new ServerJMSBytesMessage(newMessage(org.apache.activemq.artemis.api.core.Message.BYTES_TYPE), 0);
|
||||
ServerJMSBytesMessage message = new ServerJMSBytesMessage(newMessage(org.apache.activemq.artemis.api.core.Message.BYTES_TYPE));
|
||||
|
||||
if (compression) {
|
||||
// TODO
|
||||
|
@ -863,7 +588,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
}
|
||||
|
||||
private ServerJMSMapMessage createMapMessage(boolean compression) {
|
||||
ServerJMSMapMessage message = new ServerJMSMapMessage(newMessage(org.apache.activemq.artemis.api.core.Message.MAP_TYPE), 0);
|
||||
ServerJMSMapMessage message = new ServerJMSMapMessage(newMessage(org.apache.activemq.artemis.api.core.Message.MAP_TYPE));
|
||||
|
||||
if (compression) {
|
||||
// TODO
|
||||
|
@ -877,7 +602,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
}
|
||||
|
||||
private ServerJMSStreamMessage createStreamMessage(boolean compression) {
|
||||
ServerJMSStreamMessage message = new ServerJMSStreamMessage(newMessage(org.apache.activemq.artemis.api.core.Message.STREAM_TYPE), 0);
|
||||
ServerJMSStreamMessage message = new ServerJMSStreamMessage(newMessage(org.apache.activemq.artemis.api.core.Message.STREAM_TYPE));
|
||||
|
||||
if (compression) {
|
||||
// TODO
|
||||
|
@ -895,7 +620,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
}
|
||||
|
||||
private ServerJMSObjectMessage createObjectMessage(Serializable payload, boolean compression) {
|
||||
ServerJMSObjectMessage result = AMQPMessageSupport.createObjectMessage(idGenerator);
|
||||
ServerJMSObjectMessage result = AMQPMessageSupport.createObjectMessage(0);
|
||||
|
||||
if (compression) {
|
||||
// TODO
|
||||
|
@ -922,7 +647,7 @@ public class JMSMappingOutboundTransformerTest {
|
|||
}
|
||||
|
||||
private ServerJMSTextMessage createTextMessage(String text, boolean compression) {
|
||||
ServerJMSTextMessage result = AMQPMessageSupport.createTextMessage(idGenerator);
|
||||
ServerJMSTextMessage result = AMQPMessageSupport.createTextMessage(0);
|
||||
|
||||
if (compression) {
|
||||
// TODO
|
||||
|
@ -943,8 +668,8 @@ public class JMSMappingOutboundTransformerTest {
|
|||
}
|
||||
}
|
||||
|
||||
private ServerMessageImpl newMessage(byte messageType) {
|
||||
ServerMessageImpl message = new ServerMessageImpl(idGenerator.generateID(), 512);
|
||||
private CoreMessage newMessage(byte messageType) {
|
||||
CoreMessage message = new CoreMessage(0, 512);
|
||||
message.setType(messageType);
|
||||
((ResetLimitWrappedActiveMQBuffer) message.getBodyBuffer()).setMessage(null);
|
||||
return message;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue