This commit is contained in:
Martyn Taylor 2017-03-06 11:53:15 +00:00
commit a353da0caf
307 changed files with 7221 additions and 11173 deletions

View File

@ -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);

View File

@ -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()));

View File

@ -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);
}

View File

@ -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++) {

View File

@ -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);
}

View File

@ -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.

View File

@ -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
*

View File

@ -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;

View File

@ -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);

View File

@ -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";
}

View File

@ -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++) {

View File

@ -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

View File

@ -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) {

View File

@ -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);
}

View File

@ -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)
*/

View File

@ -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;
}
}

View File

@ -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();
}

View File

@ -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;
}
}

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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) {

View File

@ -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();

View File

@ -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());

View File

@ -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());

View File

@ -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);
}
}

View File

@ -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();
}

View File

@ -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) {

View File

@ -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);

View File

@ -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);
}

View File

@ -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.

View File

@ -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;
}
}

View File

@ -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();
}

View File

@ -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,

View File

@ -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();

View File

@ -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) {

View File

@ -353,6 +353,7 @@ public class RemotingConnectionImpl extends AbstractRemotingConnection implement
}
dataReceived = true;
doBufferReceived(packet);
super.bufferReceived(connectionID, buffer);

View File

@ -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);
}
}
}

View File

@ -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);
}

View File

@ -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

View File

@ -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;

View File

@ -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;
}

View File

@ -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

View File

@ -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

View File

@ -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());
}
}

View File

@ -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,

View File

@ -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);
}
}

View File

@ -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>

View File

@ -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 {
}

View File

@ -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);
}

View File

@ -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());
}

View File

@ -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;
}
}

View File

@ -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();

View File

@ -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) {

View File

@ -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;
}
}

View File

@ -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;
/**

View File

@ -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());

View File

@ -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();

View File

@ -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();

View File

@ -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));

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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) {

View File

@ -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) {

View File

@ -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,

View File

@ -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;

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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;
}
}

View File

@ -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);
}
}
}

View File

@ -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.");
}
}

View File

@ -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);
}
}
}

View File

@ -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

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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

View File

@ -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 {

View File

@ -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 ------------------------------------

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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();
}
}
}

View File

@ -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()

View File

@ -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();

View File

@ -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();

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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
}
}
}

View File

@ -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 {

View File

@ -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;

View File

@ -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 ---------------------------------------------------//

View File

@ -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);
}
}

View File

@ -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