mirror of
https://github.com/apache/activemq-artemis.git
synced 2025-03-06 09:20:07 +00:00
This closes #2986
This commit is contained in:
commit
11a00bea5f
@ -25,7 +25,7 @@ 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.core.message.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.apache.activemq.artemis.reader.TextMessageUtil;
|
||||
|
||||
@ -80,14 +80,14 @@ public class XMLMessageExporter {
|
||||
|
||||
public void printLargeMessageBody(LargeServerMessage message) throws XMLStreamException {
|
||||
xmlWriter.writeAttribute(XmlDataConstants.MESSAGE_IS_LARGE, Boolean.TRUE.toString());
|
||||
LargeBodyEncoder encoder = null;
|
||||
LargeBodyReader encoder = null;
|
||||
|
||||
try {
|
||||
encoder = message.toCore().getBodyEncoder();
|
||||
encoder = message.toMessage().toCore().getLargeBodyReader();
|
||||
encoder.open();
|
||||
long totalBytesWritten = 0;
|
||||
int bufferSize;
|
||||
long bodySize = encoder.getLargeBodySize();
|
||||
long bodySize = encoder.getSize();
|
||||
ByteBuffer buffer = null;
|
||||
for (long i = 0; i < bodySize; i += LARGE_MESSAGE_CHUNK_SIZE) {
|
||||
long remainder = bodySize - totalBytesWritten;
|
||||
@ -97,7 +97,7 @@ public class XMLMessageExporter {
|
||||
bufferSize = (int) remainder;
|
||||
}
|
||||
buffer = acquireHeapBodyBuffer(buffer, bufferSize);
|
||||
encoder.encode(buffer);
|
||||
encoder.readInto(buffer);
|
||||
xmlWriter.writeCData(XmlDataExporterUtil.encode(buffer.array()));
|
||||
totalBytesWritten += bufferSize;
|
||||
}
|
||||
|
@ -1,3 +1,3 @@
|
||||
|
||||
<!-- AMQP Acceptor. Listens on default AMQP port for AMQP traffic.-->
|
||||
<acceptor name="amqp">tcp://${host}:${amqp.port}?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;protocols=AMQP;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
|
||||
<acceptor name="amqp">tcp://${host}:${amqp.port}?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;protocols=AMQP;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpMinLargeMessageSize=102400;amqpDuplicateDetection=true</acceptor>
|
||||
|
@ -83,13 +83,16 @@ ${jdbc}
|
||||
<!-- amqpLowCredits: The server will send the # credits specified at amqpCredits at this low mark -->
|
||||
<!-- amqpDuplicateDetection: If you are not using duplicate detection, set this to false
|
||||
as duplicate detection requires applicationProperties to be parsed on the server. -->
|
||||
<!-- amqpMinLargeMessageSize: Determines how many bytes are considered large, so we start using files to hold their data.
|
||||
default: 102400, -1 would mean to disable large mesasge control -->
|
||||
|
||||
<!-- Note: If an acceptor needs to be compatible with HornetQ and/or Artemis 1.x clients add
|
||||
"anycastPrefix=jms.queue.;multicastPrefix=jms.topic." to the acceptor url.
|
||||
See https://issues.apache.org/jira/browse/ARTEMIS-1644 for more information. -->
|
||||
|
||||
|
||||
<!-- Acceptor for every supported protocol -->
|
||||
<acceptor name="artemis">tcp://${host}:${default.port}?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
|
||||
<acceptor name="artemis">tcp://${host}:${default.port}?tcpSendBufferSize=1048576;tcpReceiveBufferSize=1048576;amqpMinLargeMessageSize=102400;protocols=CORE,AMQP,STOMP,HORNETQ,MQTT,OPENWIRE;useEpoll=true;amqpCredits=1000;amqpLowCredits=300;amqpDuplicateDetection=true</acceptor>
|
||||
${amqp-acceptor}${stomp-acceptor}${hornetq-acceptor}${mqtt-acceptor}
|
||||
</acceptors>
|
||||
|
||||
|
@ -14,7 +14,7 @@
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.activemq.artemis.core.message.impl;
|
||||
package org.apache.activemq.artemis.core.persistence;
|
||||
|
||||
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
@ -19,22 +19,19 @@ package org.apache.activemq.artemis.core.persistence;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
|
||||
public interface Persister<T extends Object, A> {
|
||||
public interface Persister<T extends Object> {
|
||||
|
||||
/**
|
||||
* This is to be used to store the protocol-id on Messages.
|
||||
* Messages are stored on their bare format.
|
||||
* The protocol manager will be responsible to code or decode messages.
|
||||
* The caveat here is that the first short-sized bytes need to be this constant.
|
||||
*/
|
||||
default byte getID() {
|
||||
return (byte) 0;
|
||||
}
|
||||
|
||||
/** This is to be used to store the protocol-id on Messages.
|
||||
* Messages are stored on their bare format.
|
||||
* The protocol manager will be responsible to code or decode messages.
|
||||
* The caveat here is that the first short-sized bytes need to be this constant. */
|
||||
byte getID();
|
||||
|
||||
int getEncodeSize(T record);
|
||||
|
||||
void encode(ActiveMQBuffer buffer, T record);
|
||||
|
||||
T decode(ActiveMQBuffer buffer, A arg);
|
||||
T decode(ActiveMQBuffer buffer, T record, CoreMessageObjectPools pool);
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,41 @@
|
||||
/*
|
||||
* 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.persistence;
|
||||
|
||||
|
||||
|
||||
/** this is a list for all the persisters
|
||||
The sole purpose of this is to make sure these IDs will not be duplicate
|
||||
so we know where to find IDs.
|
||||
*/
|
||||
|
||||
public class PersisterIDs {
|
||||
|
||||
public static final int MAX_PERSISTERS = 4;
|
||||
|
||||
public static final byte CoreLargeMessagePersister_ID = (byte)0;
|
||||
|
||||
public static final byte CoreMessagePersister_ID = (byte)1;
|
||||
|
||||
public static final byte AMQPMessagePersister_ID = (byte)2;
|
||||
|
||||
public static final byte AMQPMessagePersisterV2_ID = (byte)3;
|
||||
|
||||
public static final byte AMQPLargeMessagePersister_ID = (byte)4;
|
||||
|
||||
}
|
@ -21,7 +21,7 @@ import java.io.InputStream;
|
||||
import java.util.Map;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
|
||||
/**
|
||||
@ -29,7 +29,7 @@ import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
*/
|
||||
public interface ICoreMessage extends Message {
|
||||
|
||||
LargeBodyEncoder getBodyEncoder() throws ActiveMQException;
|
||||
LargeBodyReader getLargeBodyReader() throws ActiveMQException;
|
||||
|
||||
int getHeadersAndPropertiesEncodeSize();
|
||||
|
||||
|
@ -23,7 +23,7 @@ import java.util.Set;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
|
||||
/**
|
||||
@ -182,6 +182,9 @@ public interface Message {
|
||||
/** The message will contain another message persisted through {@link org.apache.activemq.artemis.spi.core.protocol.EmbedMessageUtil}*/
|
||||
byte EMBEDDED_TYPE = 7;
|
||||
|
||||
/** This is to embedd Large Messages from other protocol */
|
||||
byte LARGE_EMBEDDED_TYPE = 8;
|
||||
|
||||
default void clearInternalProperties() {
|
||||
// only on core
|
||||
}
|
||||
@ -255,13 +258,6 @@ public interface Message {
|
||||
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();
|
||||
|
||||
default SimpleString getGroupID() {
|
||||
return null;
|
||||
}
|
||||
@ -295,8 +291,6 @@ public interface Message {
|
||||
|
||||
Message setReplyTo(SimpleString address);
|
||||
|
||||
Message setContext(RefCountMessageListener context);
|
||||
|
||||
/** The buffer will belong to this message, until release is called. */
|
||||
Message setBuffer(ByteBuf buffer);
|
||||
|
||||
@ -394,7 +388,7 @@ public interface Message {
|
||||
*/
|
||||
Message setDurable(boolean durable);
|
||||
|
||||
Persister<Message, CoreMessageObjectPools> getPersister();
|
||||
Persister<Message> getPersister();
|
||||
|
||||
String getAddress();
|
||||
|
||||
@ -671,13 +665,29 @@ public interface Message {
|
||||
|
||||
int getRefCount();
|
||||
|
||||
int incrementRefCount() throws Exception;
|
||||
int getUsage();
|
||||
|
||||
int decrementRefCount() throws Exception;
|
||||
int getDurableCount();
|
||||
|
||||
int incrementDurableRefCount();
|
||||
/** this method indicates usage by components such as large message or page cache.
|
||||
* This method will cause large messages to be held longer after the ack happened for instance.
|
||||
*/
|
||||
int usageUp();
|
||||
|
||||
int decrementDurableRefCount();
|
||||
/**
|
||||
* @see #usageUp()
|
||||
* @return
|
||||
* @throws Exception
|
||||
*/
|
||||
int usageDown();
|
||||
|
||||
int refUp();
|
||||
|
||||
int refDown();
|
||||
|
||||
int durableUp();
|
||||
|
||||
int durableDown();
|
||||
|
||||
/**
|
||||
* @return Returns the message in Map form, useful when encoding to JSON
|
||||
|
@ -19,66 +19,155 @@ package org.apache.activemq.artemis.api.core;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
|
||||
|
||||
public abstract class RefCountMessage implements Message {
|
||||
// import org.apache.activemq.artemis.utils.collections.ConcurrentHashSet; -- #ifdef DEBUG
|
||||
|
||||
public class RefCountMessage {
|
||||
|
||||
private static final AtomicIntegerFieldUpdater<RefCountMessage> DURABLE_REF_COUNT_UPDATER = AtomicIntegerFieldUpdater.newUpdater(RefCountMessage.class, "durableRefCount");
|
||||
private static final AtomicIntegerFieldUpdater<RefCountMessage> REF_COUNT_UPDATER = AtomicIntegerFieldUpdater.newUpdater(RefCountMessage.class, "refCount");
|
||||
private static final AtomicIntegerFieldUpdater<RefCountMessage> REF_USAGE_UPDATER = AtomicIntegerFieldUpdater.newUpdater(RefCountMessage.class, "usageCount");
|
||||
|
||||
private volatile int durableRefCount = 0;
|
||||
|
||||
private volatile int refCount = 0;
|
||||
|
||||
private RefCountMessageListener context;
|
||||
private volatile int usageCount = 0;
|
||||
|
||||
@Override
|
||||
public Message setContext(RefCountMessageListener context) {
|
||||
this.context = context;
|
||||
return this;
|
||||
}
|
||||
private volatile boolean fired = false;
|
||||
|
||||
@Override
|
||||
public RefCountMessageListener getContext() {
|
||||
return context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getRefCount() {
|
||||
return refCount;
|
||||
return REF_COUNT_UPDATER.get(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementRefCount() throws Exception {
|
||||
int count = REF_COUNT_UPDATER.incrementAndGet(this);
|
||||
if (context != null) {
|
||||
context.nonDurableUp(this, count);
|
||||
public int getUsage() {
|
||||
return REF_USAGE_UPDATER.get(this);
|
||||
}
|
||||
|
||||
public int getDurableCount() {
|
||||
return DURABLE_REF_COUNT_UPDATER.get(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* in certain cases the large message is copied from another LargeServerMessage
|
||||
* and the ref count needs to be on that place.
|
||||
*/
|
||||
private volatile RefCountMessage parentRef;
|
||||
|
||||
public RefCountMessage getParentRef() {
|
||||
return parentRef;
|
||||
}
|
||||
// I am usually against keeping commented out code
|
||||
// However this is very useful for me to debug referencing counting.
|
||||
// Uncomment out anything between #ifdef DEBUG and #endif
|
||||
|
||||
// #ifdef DEBUG -- comment out anything before endif if you want to debug REFERENCE COUNTS
|
||||
//final ConcurrentHashSet<Exception> upSet = new ConcurrentHashSet<>();
|
||||
// #endif
|
||||
|
||||
private void onUp() {
|
||||
// #ifdef DEBUG -- comment out anything before endif if you want to debug REFERENCE COUNTS
|
||||
// upSet.add(new Exception("upEvent(" + debugString() + ")"));
|
||||
// #endif
|
||||
}
|
||||
|
||||
private void onDown() {
|
||||
// #ifdef DEBUG -- comment out anything before endif if you want to debug REFERENCE COUNTS
|
||||
// upSet.add(new Exception("upEvent(" + debugString() + ")"));
|
||||
// #endif
|
||||
if (getRefCount() <= 0 && getUsage() <= 0 && getDurableCount() <= 0 && !fired) {
|
||||
|
||||
debugRefs();
|
||||
fired = true;
|
||||
releaseComplete();
|
||||
}
|
||||
}
|
||||
/**
|
||||
*
|
||||
* This method will be useful if you remove commented out code around #ifdef AND #endif COMMENTS
|
||||
* */
|
||||
public final void debugRefs() {
|
||||
// #ifdef DEBUG -- comment out anything before endif if you want to debug REFERENCE COUNTS
|
||||
// try {
|
||||
// System.err.println("************************************************************************************************************************");
|
||||
// System.err.println("Printing refcounts for " + debugString() + " this = " + this);
|
||||
// for (Exception e : upSet) {
|
||||
// e.printStackTrace();
|
||||
// }
|
||||
// System.err.println("************************************************************************************************************************");
|
||||
// } catch (Throwable e) {
|
||||
// e.printStackTrace();
|
||||
// }
|
||||
// #ifdef DEBUG -- comment out anything before endif if you want to debug REFERENCE COUNTS
|
||||
}
|
||||
|
||||
|
||||
|
||||
public String debugString() {
|
||||
return "refCount=" + getRefCount() + ", durableRefCount=" + getDurableCount() + ", usageCount=" + getUsage() + ", parentRef=" + this.parentRef;
|
||||
}
|
||||
public void setParentRef(RefCountMessage origin) {
|
||||
// if copy of a copy.. just go to the parent:
|
||||
if (origin.getParentRef() != null) {
|
||||
this.parentRef = origin.getParentRef();
|
||||
} else {
|
||||
this.parentRef = origin;
|
||||
}
|
||||
}
|
||||
|
||||
protected void releaseComplete() {
|
||||
}
|
||||
|
||||
public int usageUp() {
|
||||
if (parentRef != null) {
|
||||
return parentRef.usageUp();
|
||||
}
|
||||
int count = REF_USAGE_UPDATER.incrementAndGet(this);
|
||||
onUp();
|
||||
return count;
|
||||
}
|
||||
public int usageDown() {
|
||||
if (parentRef != null) {
|
||||
return parentRef.usageDown();
|
||||
}
|
||||
int count = REF_USAGE_UPDATER.decrementAndGet(this);
|
||||
onDown();
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementDurableRefCount() {
|
||||
public int durableUp() {
|
||||
if (parentRef != null) {
|
||||
return parentRef.durableUp();
|
||||
}
|
||||
int count = DURABLE_REF_COUNT_UPDATER.incrementAndGet(this);
|
||||
if (context != null) {
|
||||
context.durableUp(this, count);
|
||||
}
|
||||
onUp();
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementDurableRefCount() {
|
||||
public int durableDown() {
|
||||
if (parentRef != null) {
|
||||
return parentRef.durableDown();
|
||||
}
|
||||
int count = DURABLE_REF_COUNT_UPDATER.decrementAndGet(this);
|
||||
if (context != null) {
|
||||
context.durableDown(this, count);
|
||||
}
|
||||
onDown();
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementRefCount() throws Exception {
|
||||
int count = REF_COUNT_UPDATER.decrementAndGet(this);
|
||||
if (context != null) {
|
||||
context.nonDurableDown(this, count);
|
||||
public int refDown() {
|
||||
if (parentRef != null) {
|
||||
return parentRef.refDown();
|
||||
}
|
||||
int count = REF_COUNT_UPDATER.decrementAndGet(this);
|
||||
onDown();
|
||||
return count;
|
||||
}
|
||||
|
||||
public int refUp() {
|
||||
if (parentRef != null) {
|
||||
return parentRef.refUp();
|
||||
}
|
||||
int count = REF_COUNT_UPDATER.incrementAndGet(this);
|
||||
onUp();
|
||||
return count;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -25,7 +25,14 @@ public interface RefCountMessageListener {
|
||||
|
||||
void durableDown(Message message, int durableCount);
|
||||
|
||||
void nonDurableUp(Message message, int nonDurableCoun);
|
||||
void refUp(Message message, int nonDurableCoun);
|
||||
|
||||
void nonDurableDown(Message message, int nonDurableCoun);
|
||||
void refDown(Message message, int nonDurableCoun);
|
||||
|
||||
default void usageUp(Message message, int usageCount) {
|
||||
}
|
||||
|
||||
default void usageDown(Message message, int usageCount) {
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -177,7 +177,7 @@ public final class ClientLargeMessageImpl extends ClientMessageImpl implements C
|
||||
}
|
||||
|
||||
public void retrieveExistingData(ClientMessageInternal clMessage) {
|
||||
this.messageID = clMessage.getMessageID();
|
||||
this.internalSetMessageID(clMessage.getMessageID());
|
||||
this.address = clMessage.getAddressSimpleString();
|
||||
this.setUserID(clMessage.getUserID());
|
||||
this.setFlowControlSize(clMessage.getFlowControlSize());
|
||||
|
@ -26,9 +26,9 @@ 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.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.reader.MessageUtil;
|
||||
import org.apache.activemq.artemis.utils.UUID;
|
||||
|
||||
@ -235,7 +235,7 @@ public class ClientMessageImpl extends CoreMessage implements ClientMessageInter
|
||||
}
|
||||
|
||||
@Override
|
||||
public LargeBodyEncoder getBodyEncoder() throws ActiveMQException {
|
||||
public LargeBodyReader getLargeBodyReader() throws ActiveMQException {
|
||||
return new DecodingContext();
|
||||
}
|
||||
|
||||
@ -368,7 +368,7 @@ public class ClientMessageImpl extends CoreMessage implements ClientMessageInter
|
||||
return this;
|
||||
}
|
||||
|
||||
private final class DecodingContext implements LargeBodyEncoder {
|
||||
private final class DecodingContext implements LargeBodyReader {
|
||||
|
||||
private DecodingContext() {
|
||||
}
|
||||
@ -383,7 +383,7 @@ public class ClientMessageImpl extends CoreMessage implements ClientMessageInter
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLargeBodySize() {
|
||||
public long getSize() {
|
||||
if (isLargeMessage()) {
|
||||
return getBodyBuffer().writerIndex();
|
||||
} else {
|
||||
@ -392,7 +392,17 @@ public class ClientMessageImpl extends CoreMessage implements ClientMessageInter
|
||||
}
|
||||
|
||||
@Override
|
||||
public int encode(final ByteBuffer bufferRead) {
|
||||
public void position(long position) {
|
||||
buffer.readerIndex((int)position);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long position() {
|
||||
return buffer.readerIndex();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readInto(final ByteBuffer bufferRead) {
|
||||
final int remaining = bufferRead.remaining();
|
||||
buffer.readBytes(bufferRead);
|
||||
return remaining;
|
||||
|
@ -27,7 +27,7 @@ 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.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.SessionContext;
|
||||
import org.apache.activemq.artemis.utils.ActiveMQBufferInputStream;
|
||||
import org.apache.activemq.artemis.utils.DeflaterReader;
|
||||
@ -362,9 +362,9 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
||||
SendAcknowledgementHandler handler) throws ActiveMQException {
|
||||
sendInitialLargeMessageHeader(msgI, credits);
|
||||
|
||||
LargeBodyEncoder context = msgI.getBodyEncoder();
|
||||
LargeBodyReader context = msgI.getLargeBodyReader();
|
||||
|
||||
final long bodySize = context.getLargeBodySize();
|
||||
final long bodySize = context.getSize();
|
||||
context.open();
|
||||
try {
|
||||
|
||||
@ -375,7 +375,7 @@ public class ClientProducerImpl implements ClientProducerInternal {
|
||||
|
||||
final ByteBuffer bodyBuffer = ByteBuffer.allocate(chunkLength);
|
||||
|
||||
final int encodedSize = context.encode(bodyBuffer);
|
||||
final int encodedSize = context.readInto(bodyBuffer);
|
||||
|
||||
assert encodedSize == chunkLength;
|
||||
|
||||
|
@ -46,7 +46,7 @@ import org.apache.activemq.artemis.api.core.client.SendAcknowledgementHandler;
|
||||
import org.apache.activemq.artemis.api.core.client.SessionFailureListener;
|
||||
import org.apache.activemq.artemis.core.client.ActiveMQClientLogger;
|
||||
import org.apache.activemq.artemis.core.client.ActiveMQClientMessageBundle;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.remoting.FailureListener;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
|
||||
import org.apache.activemq.artemis.spi.core.remoting.ConsumerContext;
|
||||
|
@ -21,17 +21,33 @@ import java.nio.ByteBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
|
||||
/**
|
||||
* Class used to encode message body into buffers.
|
||||
* Class used to readInto message body into buffers.
|
||||
* <br>
|
||||
* Used to send large streams over the wire
|
||||
*
|
||||
* None of these methods should be caleld from Clients
|
||||
*/
|
||||
public interface LargeBodyEncoder {
|
||||
public interface LargeBodyReader {
|
||||
|
||||
/**
|
||||
* This method must not be called directly by ActiveMQ Artemis clients.
|
||||
*/
|
||||
void open() throws ActiveMQException;
|
||||
|
||||
/**
|
||||
* This method must not be called directly by ActiveMQ Artemis clients.
|
||||
*
|
||||
* This is the reading position.
|
||||
*/
|
||||
void position(long position) throws ActiveMQException;
|
||||
|
||||
/**
|
||||
* This method must not be called directly by ActiveMQ Artemis clients.
|
||||
*
|
||||
* This is the reading position.
|
||||
*/
|
||||
long position();
|
||||
|
||||
/**
|
||||
* This method must not be called directly by ActiveMQ Artemis clients.
|
||||
*/
|
||||
@ -40,10 +56,10 @@ public interface LargeBodyEncoder {
|
||||
/**
|
||||
* This method must not be called directly by ActiveMQ Artemis clients.
|
||||
*/
|
||||
int encode(ByteBuffer bufferRead) throws ActiveMQException;
|
||||
int readInto(ByteBuffer bufferRead) throws ActiveMQException;
|
||||
|
||||
/**
|
||||
* This method must not be called directly by ActiveMQ Artemis clients.
|
||||
*/
|
||||
long getLargeBodySize() throws ActiveMQException;
|
||||
long getSize() throws ActiveMQException;
|
||||
}
|
@ -39,7 +39,8 @@ import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ResetLimitWrappedActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl;
|
||||
import org.apache.activemq.artemis.reader.MessageUtil;
|
||||
@ -62,7 +63,7 @@ public class CoreMessage extends RefCountMessage implements ICoreMessage {
|
||||
// There's an integer with the number of bytes for the body
|
||||
public static final int BODY_OFFSET = DataConstants.SIZE_INT;
|
||||
|
||||
/** That is the encode for the whole message, including properties..
|
||||
/** That is the readInto for the whole message, including properties..
|
||||
it does not include the buffer for the Packet send and receive header on core protocol */
|
||||
protected ByteBuf buffer;
|
||||
|
||||
@ -130,7 +131,7 @@ public class CoreMessage extends RefCountMessage implements ICoreMessage {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Persister<Message, CoreMessageObjectPools> getPersister() {
|
||||
public Persister<Message> getPersister() {
|
||||
return CoreMessagePersister.getInstance();
|
||||
}
|
||||
|
||||
@ -237,13 +238,13 @@ public class CoreMessage extends RefCountMessage implements ICoreMessage {
|
||||
}
|
||||
|
||||
private ActiveMQBuffer getLargeMessageBuffer() throws ActiveMQException {
|
||||
LargeBodyEncoder encoder = getBodyEncoder();
|
||||
LargeBodyReader encoder = getLargeBodyReader();
|
||||
encoder.open();
|
||||
int bodySize = (int) encoder.getLargeBodySize();
|
||||
int bodySize = (int) encoder.getSize();
|
||||
final ActiveMQBuffer buffer = new ChannelBufferWrapper(UnpooledByteBufAllocator.DEFAULT.heapBuffer(bodySize));
|
||||
buffer.byteBuf().ensureWritable(bodySize);
|
||||
final ByteBuffer nioBuffer = buffer.byteBuf().internalNioBuffer(0, bodySize);
|
||||
encoder.encode(nioBuffer);
|
||||
encoder.readInto(nioBuffer);
|
||||
buffer.writerIndex(bodySize);
|
||||
encoder.close();
|
||||
return buffer;
|
||||
@ -427,7 +428,7 @@ public class CoreMessage extends RefCountMessage implements ICoreMessage {
|
||||
synchronized (other) {
|
||||
this.body = other.body;
|
||||
this.endOfBodyPosition = other.endOfBodyPosition;
|
||||
this.messageID = other.messageID;
|
||||
internalSetMessageID(other.messageID);
|
||||
this.address = other.address;
|
||||
this.type = other.type;
|
||||
this.durable = other.durable;
|
||||
@ -445,9 +446,15 @@ public class CoreMessage extends RefCountMessage implements ICoreMessage {
|
||||
}
|
||||
}
|
||||
|
||||
/** This method serves as a purpose of extension.
|
||||
* Large Message on a Core Message will have to set the messageID on the attached NewLargeMessage */
|
||||
protected void internalSetMessageID(final long messageID) {
|
||||
this.messageID = messageID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void moveHeadersAndProperties(final Message msg) {
|
||||
messageID = msg.getMessageID();
|
||||
internalSetMessageID(msg.getMessageID());
|
||||
address = msg.getAddressSimpleString();
|
||||
userID = (UUID) msg.getUserID();
|
||||
type = msg.toCore().getType();
|
||||
@ -523,7 +530,7 @@ public class CoreMessage extends RefCountMessage implements ICoreMessage {
|
||||
|
||||
@Override
|
||||
public CoreMessage setMessageID(long messageID) {
|
||||
this.messageID = messageID;
|
||||
internalSetMessageID(messageID);
|
||||
if (messageIDPosition >= 0 && validBuffer) {
|
||||
buffer.setLong(messageIDPosition, messageID);
|
||||
}
|
||||
@ -671,7 +678,7 @@ public class CoreMessage extends RefCountMessage implements ICoreMessage {
|
||||
|
||||
private void decodeHeadersAndProperties(final ByteBuf buffer, boolean lazyProperties, CoreMessageObjectPools pools) {
|
||||
messageIDPosition = buffer.readerIndex();
|
||||
messageID = buffer.readLong();
|
||||
internalSetMessageID(buffer.readLong());
|
||||
|
||||
address = SimpleString.readNullableSimpleString(buffer, pools == null ? null : pools.getAddressDecoderPool());
|
||||
if (buffer.readByte() == DataConstants.NOT_NULL) {
|
||||
@ -1137,32 +1144,42 @@ public class CoreMessage extends RefCountMessage implements ICoreMessage {
|
||||
}
|
||||
|
||||
@Override
|
||||
public LargeBodyEncoder getBodyEncoder() throws ActiveMQException {
|
||||
return new DecodingContext();
|
||||
public LargeBodyReader getLargeBodyReader() throws ActiveMQException {
|
||||
return new CoreLargeBodyReaderImpl();
|
||||
}
|
||||
|
||||
private final class DecodingContext implements LargeBodyEncoder {
|
||||
private final class CoreLargeBodyReaderImpl implements LargeBodyReader {
|
||||
|
||||
private int lastPos = 0;
|
||||
|
||||
private DecodingContext() {
|
||||
private CoreLargeBodyReaderImpl() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void position(long position) throws ActiveMQException {
|
||||
lastPos = (int)position;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long position() {
|
||||
return lastPos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLargeBodySize() {
|
||||
public long getSize() {
|
||||
return buffer.writerIndex();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int encode(final ByteBuffer bufferRead) {
|
||||
public int readInto(final ByteBuffer bufferRead) {
|
||||
final int remaining = bufferRead.remaining();
|
||||
buffer.getBytes(lastPos, bufferRead);
|
||||
lastPos += remaining;
|
||||
|
@ -20,11 +20,13 @@ 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.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
|
||||
public class CoreMessagePersister implements Persister<Message, CoreMessageObjectPools> {
|
||||
public static final byte ID = 1;
|
||||
import static org.apache.activemq.artemis.core.persistence.PersisterIDs.CoreMessagePersister_ID;
|
||||
public class CoreMessagePersister implements Persister<Message> {
|
||||
public static final byte ID = CoreMessagePersister_ID;
|
||||
|
||||
private static CoreMessagePersister theInstance;
|
||||
|
||||
@ -68,8 +70,9 @@ public class CoreMessagePersister implements Persister<Message, CoreMessageObjec
|
||||
record.persist(buffer);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Message decode(ActiveMQBuffer buffer, CoreMessageObjectPools pool) {
|
||||
public Message decode(ActiveMQBuffer buffer, Message record, CoreMessageObjectPools pool) {
|
||||
// the caller must consume the first byte already, as that will be used to decide what persister (protocol) to use
|
||||
long id = buffer.readLong();
|
||||
final SimpleString address;
|
||||
@ -78,7 +81,7 @@ public class CoreMessagePersister implements Persister<Message, CoreMessageObjec
|
||||
} else {
|
||||
address = SimpleString.readNullableSimpleString(buffer.byteBuf(), pool.getAddressDecoderPool());
|
||||
}
|
||||
CoreMessage record = new CoreMessage();
|
||||
record = new CoreMessage();
|
||||
record.reloadPersistence(buffer, pool);
|
||||
record.setMessageID(id);
|
||||
record.setAddress(address);
|
||||
|
@ -25,9 +25,9 @@ import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
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.RefCountMessageListener;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.BodyEncoder;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.utils.TypedProperties;
|
||||
|
||||
@ -114,17 +114,6 @@ public class MessageInternalImpl implements MessageInternal {
|
||||
return message.getScheduledDeliveryTime();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
@Override
|
||||
public RefCountMessageListener getContext() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SimpleString getReplyTo() {
|
||||
return message.getReplyTo();
|
||||
@ -136,11 +125,6 @@ public class MessageInternalImpl implements MessageInternal {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message setContext(RefCountMessageListener context) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* The buffer will belong to this message, until release is called.
|
||||
*
|
||||
@ -248,7 +232,7 @@ public class MessageInternalImpl implements MessageInternal {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Persister<Message, CoreMessageObjectPools> getPersister() {
|
||||
public Persister<Message> getPersister() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@ -643,22 +627,42 @@ public class MessageInternalImpl implements MessageInternal {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementRefCount() throws Exception {
|
||||
public int getDurableCount() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int refUp() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementRefCount() throws Exception {
|
||||
public int refDown() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementDurableRefCount() {
|
||||
public int usageUp() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementDurableRefCount() {
|
||||
public int usageDown() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUsage() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int durableUp() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int durableDown() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
|
@ -19,7 +19,7 @@ package org.apache.activemq.artemis.core.protocol;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientLargeMessageImpl;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientMessageImpl;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.protocol.core.CoreRemotingConnection;
|
||||
import org.apache.activemq.artemis.core.protocol.core.Packet;
|
||||
import org.apache.activemq.artemis.core.protocol.core.impl.PacketDecoder;
|
||||
|
@ -27,7 +27,7 @@ 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.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.reader.TextMessageUtil;
|
||||
import org.apache.activemq.artemis.utils.UUID;
|
||||
import org.apache.activemq.artemis.utils.UUIDGenerator;
|
||||
|
@ -142,6 +142,11 @@ public class JDBCSequentialFile implements SequentialFile {
|
||||
// Do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer map(int position, long size) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete() throws IOException, InterruptedException, ActiveMQException {
|
||||
try {
|
||||
|
@ -45,6 +45,8 @@ public interface SequentialFile {
|
||||
*/
|
||||
void open(int maxIO, boolean useExecutor) throws Exception;
|
||||
|
||||
ByteBuffer map(int position, long size) throws IOException;
|
||||
|
||||
boolean fits(int size);
|
||||
|
||||
int calculateBlockStart(int position) throws Exception;
|
||||
|
@ -78,6 +78,11 @@ public class AIOSequentialFile extends AbstractSequentialFile {
|
||||
this.aioFactory = factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer map(int position, long size) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isOpen() {
|
||||
return opened;
|
||||
|
@ -418,6 +418,11 @@ final class MappedSequentialFile implements SequentialFile {
|
||||
SequentialFile.appendTo(file.toPath(), dstFile.getJavaFile().toPath());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer map(int position, long size) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public void setTimedBuffer(TimedBuffer buffer) {
|
||||
|
@ -78,6 +78,11 @@ final class TimedSequentialFile implements SequentialFile {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer map(int position, long size) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int calculateBlockStart(int position) throws Exception {
|
||||
return this.sequentialFile.calculateBlockStart(position);
|
||||
|
@ -18,12 +18,17 @@ package org.apache.activemq.artemis.core.io.nio;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.io.StringWriter;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
@ -42,6 +47,8 @@ import org.apache.activemq.artemis.utils.Env;
|
||||
|
||||
public class NIOSequentialFile extends AbstractSequentialFile {
|
||||
|
||||
private static final boolean DEBUG_OPENS = false;
|
||||
|
||||
/* This value has been tuned just to reduce the memory footprint
|
||||
of read/write of the whole file size: given that this value
|
||||
is > 8192, RandomAccessFile JNI code will use malloc/free instead
|
||||
@ -91,6 +98,45 @@ public class NIOSequentialFile extends AbstractSequentialFile {
|
||||
open(maxIO, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer map(int position, long size) throws IOException {
|
||||
return channel.map(FileChannel.MapMode.READ_ONLY, 0, size);
|
||||
}
|
||||
|
||||
public static void clearDebug() {
|
||||
counters.clear();
|
||||
}
|
||||
|
||||
public static void printDebug() {
|
||||
for (Map.Entry<String, AtomicInteger> entry : counters.entrySet()) {
|
||||
System.out.println(entry.getValue() + " " + entry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
public static AtomicInteger getDebugCounter(Exception location) {
|
||||
StringWriter writer = new StringWriter();
|
||||
PrintWriter printWriter = new PrintWriter(writer);
|
||||
location.printStackTrace(printWriter);
|
||||
|
||||
String strLocation = writer.toString();
|
||||
|
||||
return getDebugCounter(strLocation);
|
||||
}
|
||||
|
||||
public static AtomicInteger getDebugCounter(String strLocation) {
|
||||
AtomicInteger value = counters.get(strLocation);
|
||||
if (value == null) {
|
||||
value = new AtomicInteger(0);
|
||||
AtomicInteger oldvalue = counters.putIfAbsent(strLocation, value);
|
||||
if (oldvalue != null) {
|
||||
value = oldvalue;
|
||||
}
|
||||
}
|
||||
|
||||
return value;
|
||||
}
|
||||
|
||||
private static Map<String, AtomicInteger> counters = new ConcurrentHashMap<>();
|
||||
@Override
|
||||
public void open(final int maxIO, final boolean useExecutor) throws IOException {
|
||||
try {
|
||||
@ -99,6 +145,11 @@ public class NIOSequentialFile extends AbstractSequentialFile {
|
||||
channel = rfile.getChannel();
|
||||
|
||||
fileSize = channel.size();
|
||||
|
||||
if (DEBUG_OPENS) {
|
||||
getDebugCounter(new Exception("open")).incrementAndGet();
|
||||
getDebugCounter("open").incrementAndGet();
|
||||
}
|
||||
} catch (ClosedChannelException e) {
|
||||
throw e;
|
||||
} catch (IOException e) {
|
||||
@ -152,6 +203,11 @@ public class NIOSequentialFile extends AbstractSequentialFile {
|
||||
public synchronized void close(boolean waitSync) throws IOException, InterruptedException, ActiveMQException {
|
||||
super.close();
|
||||
|
||||
if (DEBUG_OPENS) {
|
||||
getDebugCounter(new Exception("Close")).incrementAndGet();
|
||||
getDebugCounter("close").incrementAndGet();
|
||||
}
|
||||
|
||||
try {
|
||||
try {
|
||||
if (channel != null) {
|
||||
|
@ -18,17 +18,23 @@
|
||||
package org.apache.activemq.artemis.core.journal;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
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, EncodingSupport> {
|
||||
public class EncoderPersister implements Persister<EncodingSupport> {
|
||||
|
||||
private static final EncoderPersister theInstance = new EncoderPersister();
|
||||
|
||||
private EncoderPersister() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getID() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
public static EncoderPersister getInstance() {
|
||||
return theInstance;
|
||||
}
|
||||
@ -44,7 +50,7 @@ public class EncoderPersister implements Persister<EncodingSupport, EncodingSupp
|
||||
}
|
||||
|
||||
@Override
|
||||
public EncodingSupport decode(ActiveMQBuffer buffer, EncodingSupport record) {
|
||||
public EncodingSupport decode(ActiveMQBuffer buffer, EncodingSupport record, CoreMessageObjectPools pools) {
|
||||
record.decode(buffer);
|
||||
return record;
|
||||
}
|
||||
|
@ -0,0 +1,373 @@
|
||||
/*
|
||||
* 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 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.Message;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFile;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.journal.LargeBody;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.journal.LargeServerMessageImpl;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.TLSEncode;
|
||||
import org.apache.activemq.artemis.utils.collections.TypedProperties;
|
||||
import org.apache.qpid.proton.amqp.messaging.Header;
|
||||
import org.apache.qpid.proton.codec.DecoderImpl;
|
||||
import org.apache.qpid.proton.codec.ReadableBuffer;
|
||||
import org.apache.qpid.proton.codec.TypeConstructor;
|
||||
|
||||
public class AMQPLargeMessage extends AMQPMessage implements LargeServerMessage {
|
||||
|
||||
@Override
|
||||
public ICoreMessage toCore(CoreMessageObjectPools coreMessageObjectPools) {
|
||||
LargeBodyReader reader = largeBody.getLargeBodyReader();
|
||||
|
||||
try {
|
||||
long size = reader.getSize();
|
||||
if (size > Integer.MAX_VALUE) {
|
||||
throw new RuntimeException("AMQP Large Message Body is too large to be converted into core");
|
||||
}
|
||||
byte[] buffer = new byte[(int)size];
|
||||
ByteBuffer wrapbuffer = ByteBuffer.wrap(buffer);
|
||||
|
||||
reader.open();
|
||||
reader.readInto(wrapbuffer);
|
||||
|
||||
AMQPStandardMessage standardMessage = new AMQPStandardMessage(messageFormat, buffer, extraProperties, coreMessageObjectPools);
|
||||
standardMessage.setMessageID(messageID);
|
||||
return standardMessage.toCore();
|
||||
} catch (Exception e) {
|
||||
logger.warn(e.getMessage(), e);
|
||||
throw new RuntimeException(e.getMessage(), e);
|
||||
} finally {
|
||||
try {
|
||||
reader.close();
|
||||
} catch (Exception e) {
|
||||
// unexpected to happen, but possible, nothing else we can do beyond logging at this point
|
||||
// if we wanted to add anything it would be a critical failure but it would be a heavy refactoring
|
||||
// to bring the bits and listeners here for little benefit
|
||||
// the possibility of this happening involves losing the storage device which will lead to other errors anyway
|
||||
logger.warn(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
private final LargeBody largeBody;
|
||||
/**
|
||||
* We control durability on a separate property here, as we need to know if it's durable ahead of the header parsing.
|
||||
* This will be the case when restarting a server
|
||||
*/
|
||||
private Boolean fileDurable;
|
||||
|
||||
private volatile AmqpReadableBuffer parsingData;
|
||||
|
||||
private final StorageManager storageManager;
|
||||
|
||||
public AMQPLargeMessage(long id,
|
||||
long messageFormat,
|
||||
TypedProperties extraProperties,
|
||||
CoreMessageObjectPools coreMessageObjectPools,
|
||||
StorageManager storageManager) {
|
||||
super(messageFormat, extraProperties, coreMessageObjectPools);
|
||||
this.setMessageID(id);
|
||||
largeBody = new LargeBody(this, storageManager);
|
||||
this.storageManager = storageManager;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copy constructor
|
||||
*/
|
||||
private AMQPLargeMessage(final AMQPLargeMessage copy,
|
||||
final SequentialFile fileCopy,
|
||||
final long newID) {
|
||||
super(copy);
|
||||
largeBody = new LargeBody(this, copy.largeBody.getStorageManager(), fileCopy);
|
||||
largeBody.setBodySize(copy.largeBody.getStoredBodySize());
|
||||
this.storageManager = copy.largeBody.getStorageManager();
|
||||
setMessageID(newID);
|
||||
}
|
||||
|
||||
public void openLargeMessage() throws Exception {
|
||||
this.parsingData = new AmqpReadableBuffer(largeBody.map());
|
||||
}
|
||||
|
||||
public void closeLargeMessage() throws Exception {
|
||||
largeBody.releaseResources(false);
|
||||
parsingData.freeDirectBuffer();
|
||||
parsingData = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishParse() throws Exception {
|
||||
openLargeMessage();
|
||||
try {
|
||||
this.ensureMessageDataScanned();
|
||||
parsingData.rewind();
|
||||
lazyDecodeApplicationProperties();
|
||||
} finally {
|
||||
closeLargeMessage();
|
||||
}
|
||||
}
|
||||
|
||||
public void setFileDurable(boolean value) {
|
||||
this.fileDurable = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageManager getStorageManager() {
|
||||
return largeBody.getStorageManager();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final boolean isDurable() {
|
||||
if (fileDurable != null) {
|
||||
return fileDurable.booleanValue();
|
||||
} else {
|
||||
return super.isDurable();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer getData() {
|
||||
if (parsingData == null) {
|
||||
throw new RuntimeException("AMQP Large Message is not open");
|
||||
}
|
||||
|
||||
return parsingData;
|
||||
}
|
||||
|
||||
protected void parseHeader(ReadableBuffer buffer) {
|
||||
|
||||
DecoderImpl decoder = TLSEncode.getDecoder();
|
||||
decoder.setBuffer(buffer);
|
||||
|
||||
try {
|
||||
int constructorPos = buffer.position();
|
||||
TypeConstructor<?> constructor = decoder.readConstructor();
|
||||
if (Header.class.equals(constructor.getTypeClass())) {
|
||||
header = (Header) constructor.readValue();
|
||||
if (header.getTtl() != null) {
|
||||
expiration = System.currentTimeMillis() + header.getTtl().intValue();
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
decoder.setBuffer(null);
|
||||
buffer.rewind();
|
||||
}
|
||||
}
|
||||
|
||||
public void addBytes(ReadableBuffer data) throws Exception {
|
||||
|
||||
// We need to parse the header on the first add,
|
||||
// as it will contain information if the message is durable or not
|
||||
if (header == null && largeBody.getStoredBodySize() <= 0) {
|
||||
parseHeader(data);
|
||||
}
|
||||
|
||||
if (data.hasArray() && data.remaining() == data.array().length) {
|
||||
//System.out.println("Received " + data.array().length + "::" + ByteUtil.formatGroup(ByteUtil.bytesToHex(data.array()), 8, 16));
|
||||
largeBody.addBytes(data.array());
|
||||
} else {
|
||||
byte[] bytes = new byte[data.remaining()];
|
||||
data.get(bytes);
|
||||
//System.out.println("Finishing " + bytes.length + ByteUtil.formatGroup(ByteUtil.bytesToHex(bytes), 8, 16));
|
||||
largeBody.addBytes(bytes);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer getSendBuffer(int deliveryCount) {
|
||||
return getData().rewind();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message toMessage() {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBytes(byte[] bytes) throws Exception {
|
||||
largeBody.addBytes(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBytes(ActiveMQBuffer bytes) throws Exception {
|
||||
largeBody.addBytes(bytes);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPaged() {
|
||||
largeBody.setPaged();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void releaseResources(boolean sync) {
|
||||
largeBody.releaseResources(sync);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteFile() throws Exception {
|
||||
largeBody.deleteFile();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SequentialFile getAppendFile() throws ActiveMQException {
|
||||
return largeBody.getAppendFile();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isLargeMessage() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LargeBodyReader getLargeBodyReader() {
|
||||
return largeBody.getLargeBodyReader();
|
||||
}
|
||||
|
||||
@Override
|
||||
public LargeBody getLargeBody() {
|
||||
return largeBody;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearPendingRecordID() {
|
||||
largeBody.clearPendingRecordID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPendingRecord() {
|
||||
return largeBody.hasPendingRecord();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPendingRecordID(long pendingRecordID) {
|
||||
largeBody.setPendingRecordID(pendingRecordID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPendingRecordID() {
|
||||
return largeBody.getPendingRecordID();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void releaseComplete() {
|
||||
largeBody.deleteFile();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message copy() {
|
||||
SequentialFile newfile = largeBody.createFile();
|
||||
AMQPLargeMessage newMessage = new AMQPLargeMessage(this, newfile, messageID);
|
||||
newMessage.setParentRef(this);
|
||||
newMessage.setFileDurable(this.isDurable());
|
||||
return newMessage;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message copy(final long newID) {
|
||||
try {
|
||||
AMQPLargeMessage copy = new AMQPLargeMessage(newID, messageFormat, null, coreMessageObjectPools, storageManager);
|
||||
copy.setDurable(this.isDurable());
|
||||
largeBody.copyInto(copy);
|
||||
copy.finishParse();
|
||||
copy.releaseResources(true);
|
||||
return copy;
|
||||
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.lareMessageErrorCopying(e, this);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public void messageChanged() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getEncodeSize() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMemoryEstimate() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persist(ActiveMQBuffer targetRecord) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPersistSize() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reloadPersistence(ActiveMQBuffer record, CoreMessageObjectPools pools) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPersistentSize() throws ActiveMQException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Persister<Message> getPersister() {
|
||||
return AMQPLargeMessagePersister.getInstance();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reencode() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void ensureDataIsValid() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void encodeMessage() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void referenceOriginalMessage(final Message original, String originalQueue) {
|
||||
|
||||
super.referenceOriginalMessage(original, originalQueue);
|
||||
|
||||
if (original instanceof LargeServerMessageImpl) {
|
||||
this.largeBody.referenceOriginalMessage(((AMQPLargeMessage) original).largeBody);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,144 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.activemq.artemis.protocol.amqp.broker;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
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.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.journal.AbstractJournalStorageManager;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.MessagePersister;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
import org.apache.activemq.artemis.utils.collections.TypedProperties;
|
||||
import org.jboss.logging.Logger;
|
||||
|
||||
import static org.apache.activemq.artemis.core.persistence.PersisterIDs.AMQPLargeMessagePersister_ID;
|
||||
|
||||
public class AMQPLargeMessagePersister extends MessagePersister {
|
||||
private static final Logger log = Logger.getLogger(AMQPLargeMessagePersister.class);
|
||||
|
||||
// We need to save the encoder ahead of time
|
||||
// as we need to know the exact size of the Encoding
|
||||
// so we store the savedBuffer on the getEncodeSize before we actually store it
|
||||
private static final ThreadLocal<ByteBuf> savedBuffer = new ThreadLocal<>();
|
||||
|
||||
public static final byte ID = AMQPLargeMessagePersister_ID;
|
||||
|
||||
public static AMQPLargeMessagePersister theInstance;
|
||||
|
||||
public static AMQPLargeMessagePersister getInstance() {
|
||||
if (theInstance == null) {
|
||||
theInstance = new AMQPLargeMessagePersister();
|
||||
}
|
||||
return theInstance;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getID() {
|
||||
return ID;
|
||||
}
|
||||
|
||||
public AMQPLargeMessagePersister() {
|
||||
super();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int getEncodeSize(Message record) {
|
||||
ByteBuf buf = getSavedEncodeBuffer(record);
|
||||
|
||||
int encodeSize = DataConstants.SIZE_BYTE + DataConstants.SIZE_INT + DataConstants.SIZE_LONG + DataConstants.SIZE_LONG + SimpleString.sizeofNullableString(record.getAddressSimpleString()) + DataConstants.SIZE_BOOLEAN + buf.writerIndex();
|
||||
|
||||
TypedProperties properties = ((AMQPMessage) record).getExtraProperties();
|
||||
|
||||
return encodeSize + (properties != null ? properties.getEncodeSize() : 0);
|
||||
}
|
||||
|
||||
private ByteBuf getSavedEncodeBuffer(Message record) {
|
||||
ByteBuf buf = savedBuffer.get();
|
||||
if (buf == null) {
|
||||
AMQPLargeMessage largeMessage = (AMQPLargeMessage)record;
|
||||
buf = PooledByteBufAllocator.DEFAULT.buffer(largeMessage.getEstimateSavedEncode());
|
||||
largeMessage.saveEncoding(buf);
|
||||
savedBuffer.set(buf);
|
||||
}
|
||||
return buf;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.writeBoolean(record.isDurable());
|
||||
buffer.writeLong(msgEncode.getMessageFormat());
|
||||
buffer.writeNullableSimpleString(record.getAddressSimpleString());
|
||||
TypedProperties properties = ((AMQPMessage) record).getExtraProperties();
|
||||
if (properties == null) {
|
||||
buffer.writeInt(0);
|
||||
} else {
|
||||
buffer.writeInt(properties.getEncodeSize());
|
||||
properties.encode(buffer.byteBuf());
|
||||
}
|
||||
|
||||
ByteBuf savedEncodeBuffer = getSavedEncodeBuffer(record);
|
||||
buffer.writeBytes(savedEncodeBuffer, 0, savedEncodeBuffer.writerIndex());
|
||||
savedEncodeBuffer.release();
|
||||
|
||||
savedBuffer.set(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message decode(ActiveMQBuffer buffer, Message record, CoreMessageObjectPools pools) {
|
||||
|
||||
long id = buffer.readLong();
|
||||
boolean durable = buffer.readBoolean();
|
||||
long format = buffer.readLong();
|
||||
SimpleString address = buffer.readNullableSimpleString();
|
||||
|
||||
int size = buffer.readInt();
|
||||
|
||||
TypedProperties properties;
|
||||
|
||||
if (size != 0) {
|
||||
properties = new TypedProperties(Message.INTERNAL_PROPERTY_NAMES_PREDICATE);
|
||||
properties.decode(buffer.byteBuf());
|
||||
} else {
|
||||
properties = null;
|
||||
}
|
||||
|
||||
AMQPLargeMessage largeMessage = new AMQPLargeMessage(id, format, properties, null, AbstractJournalStorageManager.getThreadLocal());
|
||||
|
||||
largeMessage.setFileDurable(durable);
|
||||
if (address != null) {
|
||||
largeMessage.setAddress(address);
|
||||
}
|
||||
|
||||
largeMessage.readSavedEncoding(buffer.byteBuf());
|
||||
|
||||
return largeMessage;
|
||||
}
|
||||
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -20,13 +20,14 @@ 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.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.MessagePersister;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
import static org.apache.activemq.artemis.core.persistence.PersisterIDs.AMQPMessagePersister_ID;
|
||||
|
||||
public class AMQPMessagePersister extends MessagePersister {
|
||||
|
||||
public static final byte ID = 2;
|
||||
public static final byte ID = AMQPMessagePersister_ID;
|
||||
|
||||
public static AMQPMessagePersister theInstance;
|
||||
|
||||
@ -63,7 +64,7 @@ public class AMQPMessagePersister extends MessagePersister {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message decode(ActiveMQBuffer buffer, CoreMessageObjectPools pool) {
|
||||
public Message decode(ActiveMQBuffer buffer, Message record, CoreMessageObjectPools pool) {
|
||||
long id = buffer.readLong();
|
||||
long format = buffer.readLong();
|
||||
final SimpleString address;
|
||||
@ -72,7 +73,7 @@ public class AMQPMessagePersister extends MessagePersister {
|
||||
} else {
|
||||
address = SimpleString.readNullableSimpleString(buffer.byteBuf(), pool.getAddressDecoderPool());
|
||||
}
|
||||
AMQPMessage record = new AMQPMessage(format);
|
||||
record = new AMQPStandardMessage(format);
|
||||
record.reloadPersistence(buffer, pool);
|
||||
record.setMessageID(id);
|
||||
if (address != null) {
|
||||
|
@ -19,12 +19,14 @@ 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.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
import org.apache.activemq.artemis.utils.collections.TypedProperties;
|
||||
import static org.apache.activemq.artemis.core.persistence.PersisterIDs.AMQPMessagePersisterV2_ID;
|
||||
|
||||
public class AMQPMessagePersisterV2 extends AMQPMessagePersister {
|
||||
public static final byte ID = 3;
|
||||
|
||||
public static final byte ID = AMQPMessagePersisterV2_ID;
|
||||
|
||||
public static AMQPMessagePersisterV2 theInstance;
|
||||
|
||||
@ -70,8 +72,8 @@ public class AMQPMessagePersisterV2 extends AMQPMessagePersister {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message decode(ActiveMQBuffer buffer, CoreMessageObjectPools pool) {
|
||||
AMQPMessage message = (AMQPMessage) super.decode(buffer, pool);
|
||||
public Message decode(ActiveMQBuffer buffer, Message record, CoreMessageObjectPools pool) {
|
||||
AMQPMessage message = (AMQPMessage) super.decode(buffer, record, pool);
|
||||
int size = buffer.readInt();
|
||||
|
||||
if (size != 0) {
|
||||
|
@ -29,7 +29,7 @@ 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;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.paging.PagingManager;
|
||||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
import org.apache.activemq.artemis.core.persistence.OperationContext;
|
||||
@ -106,8 +106,7 @@ public class AMQPSessionCallback implements SessionCallback {
|
||||
|
||||
private final boolean directDeliver;
|
||||
|
||||
|
||||
private CoreMessageObjectPools coreMessageObjectPools = new CoreMessageObjectPools();
|
||||
private final CoreMessageObjectPools coreMessageObjectPools = new CoreMessageObjectPools();
|
||||
|
||||
private final AddressQueryCache<AddressQueryResult> addressQueryCache = new AddressQueryCache<>();
|
||||
|
||||
@ -129,6 +128,14 @@ public class AMQPSessionCallback implements SessionCallback {
|
||||
this.directDeliver = manager.isDirectDeliver();
|
||||
}
|
||||
|
||||
public StorageManager getStorageManager() {
|
||||
return storageManager;
|
||||
}
|
||||
|
||||
public CoreMessageObjectPools getCoreMessageObjectPools() {
|
||||
return coreMessageObjectPools;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isWritable(ReadyListener callback, Object protocolContext) {
|
||||
ProtonServerSenderContext senderContext = (ProtonServerSenderContext) protocolContext;
|
||||
@ -425,6 +432,10 @@ public class AMQPSessionCallback implements SessionCallback {
|
||||
((ServerConsumer) consumer).receiveCredits(-1);
|
||||
}
|
||||
|
||||
public AMQPStandardMessage createStandardMessage(Delivery delivery, ReadableBuffer data) {
|
||||
return new AMQPStandardMessage(delivery.getMessageFormat(), data, null, coreMessageObjectPools);
|
||||
}
|
||||
|
||||
public void serverSend(final ProtonServerReceiverContext context,
|
||||
final Transaction transaction,
|
||||
final Receiver receiver,
|
||||
@ -433,7 +444,17 @@ public class AMQPSessionCallback implements SessionCallback {
|
||||
int messageFormat,
|
||||
ReadableBuffer data,
|
||||
RoutingContext routingContext) throws Exception {
|
||||
AMQPMessage message = new AMQPMessage(messageFormat, data, null, coreMessageObjectPools);
|
||||
AMQPStandardMessage message = new AMQPStandardMessage(messageFormat, data, null, coreMessageObjectPools);
|
||||
serverSend(context, transaction, receiver, delivery, address, routingContext, message);
|
||||
}
|
||||
|
||||
public void serverSend(ProtonServerReceiverContext context,
|
||||
Transaction transaction,
|
||||
Receiver receiver,
|
||||
Delivery delivery,
|
||||
SimpleString address,
|
||||
RoutingContext routingContext,
|
||||
AMQPMessage message) throws Exception {
|
||||
if (address != null) {
|
||||
message.setAddress(address);
|
||||
} else {
|
||||
|
@ -0,0 +1,262 @@
|
||||
/*
|
||||
* 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 io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.PooledByteBufAllocator;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
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.activemq.artemis.utils.collections.TypedProperties;
|
||||
import org.apache.qpid.proton.codec.EncoderImpl;
|
||||
import org.apache.qpid.proton.codec.ReadableBuffer;
|
||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||
|
||||
// see https://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-messaging-v1.0-os.html#section-message-format
|
||||
public class AMQPStandardMessage extends AMQPMessage {
|
||||
|
||||
// Buffer and state for the data backing this message.
|
||||
protected ReadableBuffer data;
|
||||
|
||||
/**
|
||||
* Creates a new {@link AMQPStandardMessage} instance from binary encoded message data.
|
||||
*
|
||||
* @param messageFormat The Message format tag given the in Transfer that carried this message
|
||||
* @param data The encoded AMQP message
|
||||
* @param extraProperties Broker specific extra properties that should be carried with this message
|
||||
*/
|
||||
public AMQPStandardMessage(long messageFormat, byte[] data, TypedProperties extraProperties) {
|
||||
this(messageFormat, data, extraProperties, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link AMQPStandardMessage} instance from binary encoded message data.
|
||||
*
|
||||
* @param messageFormat The Message format tag given the in Transfer that carried this message
|
||||
* @param data The encoded AMQP message
|
||||
* @param extraProperties Broker specific extra properties that should be carried with this message
|
||||
* @param coreMessageObjectPools Object pool used to accelerate some String operations.
|
||||
*/
|
||||
public AMQPStandardMessage(long messageFormat,
|
||||
byte[] data,
|
||||
TypedProperties extraProperties,
|
||||
CoreMessageObjectPools coreMessageObjectPools) {
|
||||
this(messageFormat, ReadableBuffer.ByteBufferReader.wrap(data), extraProperties, coreMessageObjectPools);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link AMQPStandardMessage} instance from binary encoded message data.
|
||||
*
|
||||
* @param messageFormat The Message format tag given the in Transfer that carried this message
|
||||
* @param data The encoded AMQP message in an {@link ReadableBuffer} wrapper.
|
||||
* @param extraProperties Broker specific extra properties that should be carried with this message
|
||||
* @param coreMessageObjectPools Object pool used to accelerate some String operations.
|
||||
*/
|
||||
public AMQPStandardMessage(long messageFormat,
|
||||
ReadableBuffer data,
|
||||
TypedProperties extraProperties,
|
||||
CoreMessageObjectPools coreMessageObjectPools) {
|
||||
super(messageFormat, extraProperties, coreMessageObjectPools);
|
||||
this.data = data;
|
||||
ensureMessageDataScanned();
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal constructor used for persistence reload of the message.
|
||||
* <p>
|
||||
* The message will not be usable until the persistence mechanism populates the message
|
||||
* data and triggers a parse of the message contents to fill in the message state.
|
||||
*
|
||||
* @param messageFormat The Message format tag given the in Transfer that carried this message
|
||||
*/
|
||||
AMQPStandardMessage(long messageFormat) {
|
||||
super(messageFormat);
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.apache.activemq.artemis.api.core.Message copy() {
|
||||
ensureDataIsValid();
|
||||
|
||||
ReadableBuffer view = data.duplicate().rewind();
|
||||
byte[] newData = new byte[view.remaining()];
|
||||
|
||||
// Copy the full message contents with delivery annotations as they will
|
||||
// be trimmed on send and may become useful on the broker at a later time.
|
||||
view.get(newData);
|
||||
|
||||
AMQPStandardMessage newEncode = new AMQPStandardMessage(this.messageFormat, newData, extraProperties, coreMessageObjectPools);
|
||||
newEncode.setMessageID(this.getMessageID());
|
||||
return newEncode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getEncodeSize() {
|
||||
ensureDataIsValid();
|
||||
// The encoded size will exclude any delivery annotations that are present as we will clip them.
|
||||
return data.remaining() - encodedDeliveryAnnotationsSize + getDeliveryAnnotationsForSendBufferSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ReadableBuffer getData() {
|
||||
return data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMemoryEstimate() {
|
||||
if (memoryEstimate == -1) {
|
||||
memoryEstimate = memoryOffset + (data != null ? data.capacity() : 0);
|
||||
}
|
||||
|
||||
return memoryEstimate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persist(ActiveMQBuffer targetRecord) {
|
||||
ensureDataIsValid();
|
||||
targetRecord.writeInt(internalPersistSize());
|
||||
if (data.hasArray()) {
|
||||
targetRecord.writeBytes(data.array(), data.arrayOffset(), data.remaining());
|
||||
} else {
|
||||
targetRecord.writeBytes(data.byteBuffer());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final org.apache.activemq.artemis.api.core.Message copy(long newID) {
|
||||
return copy().setMessageID(newID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPersistSize() {
|
||||
ensureDataIsValid();
|
||||
return DataConstants.SIZE_INT + internalPersistSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reloadPersistence(ActiveMQBuffer record, CoreMessageObjectPools pools) {
|
||||
int size = record.readInt();
|
||||
byte[] recordArray = new byte[size];
|
||||
record.readBytes(recordArray);
|
||||
data = ReadableBuffer.ByteBufferReader.wrap(ByteBuffer.wrap(recordArray));
|
||||
|
||||
// Message state is now that the underlying buffer is loaded, but the contents not yet scanned
|
||||
resetMessageData();
|
||||
modified = false;
|
||||
messageDataScanned = RELOAD_PERSISTENCE;
|
||||
// can happen when moved to a durable location. We must re-encode here to
|
||||
// avoid a subsequent redelivery from suddenly appearing with a durable header
|
||||
// tag when the initial delivery did not.
|
||||
if (!isDurable()) {
|
||||
setDurable(true);
|
||||
reencode();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPersistentSize() throws ActiveMQException {
|
||||
return getEncodeSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Persister<org.apache.activemq.artemis.api.core.Message> getPersister() {
|
||||
return AMQPMessagePersisterV2.getInstance();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reencode() {
|
||||
ensureMessageDataScanned();
|
||||
|
||||
// The address was updated on a message with Properties so we update them
|
||||
// for cases where there are no properties we aren't adding a properties
|
||||
// section which seems wrong but this preserves previous behavior.
|
||||
if (properties != null && address != null) {
|
||||
properties.setTo(address.toString());
|
||||
}
|
||||
|
||||
encodeMessage();
|
||||
scanMessageData();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected synchronized void ensureDataIsValid() {
|
||||
if (modified) {
|
||||
encodeMessage();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected synchronized void encodeMessage() {
|
||||
this.modified = false;
|
||||
this.messageDataScanned = NOT_SCANNED;
|
||||
int estimated = Math.max(1500, data != null ? data.capacity() + 1000 : 0);
|
||||
ByteBuf buffer = PooledByteBufAllocator.DEFAULT.heapBuffer(estimated);
|
||||
EncoderImpl encoder = TLSEncode.getEncoder();
|
||||
|
||||
try {
|
||||
NettyWritable writable = new NettyWritable(buffer);
|
||||
|
||||
encoder.setByteBuffer(writable);
|
||||
if (header != null) {
|
||||
encoder.writeObject(header);
|
||||
}
|
||||
|
||||
// We currently do not encode any delivery annotations but it is conceivable
|
||||
// that at some point they may need to be preserved, this is where that needs
|
||||
// to happen.
|
||||
|
||||
if (messageAnnotations != null) {
|
||||
encoder.writeObject(messageAnnotations);
|
||||
}
|
||||
if (properties != null) {
|
||||
encoder.writeObject(properties);
|
||||
}
|
||||
|
||||
// Whenever possible avoid encoding sections we don't need to by
|
||||
// checking if application properties where loaded or added and
|
||||
// encoding only in that case.
|
||||
if (applicationProperties != null) {
|
||||
encoder.writeObject(applicationProperties);
|
||||
|
||||
// Now raw write the remainder body and footer if present.
|
||||
if (data != null && remainingBodyPosition != VALUE_NOT_PRESENT) {
|
||||
writable.put(data.position(remainingBodyPosition));
|
||||
}
|
||||
} else if (data != null && applicationPropertiesPosition != VALUE_NOT_PRESENT) {
|
||||
// Writes out ApplicationProperties, Body and Footer in one go if present.
|
||||
writable.put(data.position(applicationPropertiesPosition));
|
||||
} else if (data != null && remainingBodyPosition != VALUE_NOT_PRESENT) {
|
||||
// No Application properties at all so raw write Body and Footer sections
|
||||
writable.put(data.position(remainingBodyPosition));
|
||||
}
|
||||
|
||||
byte[] bytes = new byte[buffer.writerIndex()];
|
||||
|
||||
buffer.readBytes(bytes);
|
||||
data = ReadableBuffer.ByteBufferReader.wrap(ByteBuffer.wrap(bytes));
|
||||
} finally {
|
||||
encoder.setByteBuffer((WritableBuffer) null);
|
||||
buffer.release();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,227 @@
|
||||
/*
|
||||
* 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.nio.charset.CharacterCodingException;
|
||||
import java.nio.charset.CharsetDecoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import io.netty.util.internal.PlatformDependent;
|
||||
import org.apache.qpid.proton.codec.ReadableBuffer;
|
||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||
|
||||
public class AmqpReadableBuffer implements ReadableBuffer {
|
||||
private ByteBuffer buffer;
|
||||
|
||||
public AmqpReadableBuffer(ByteBuffer buffer) {
|
||||
this.buffer = buffer;
|
||||
}
|
||||
|
||||
public ByteBuffer getBuffer() {
|
||||
return this.buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int capacity() {
|
||||
return this.buffer.capacity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasArray() {
|
||||
return this.buffer.hasArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] array() {
|
||||
if (this.buffer.hasArray()) {
|
||||
return this.buffer.array();
|
||||
} else {
|
||||
byte[] bytes = new byte[buffer.remaining()];
|
||||
buffer.get(bytes);
|
||||
return bytes;
|
||||
}
|
||||
}
|
||||
|
||||
public void freeDirectBuffer() {
|
||||
// releasing direct buffer created from mmap
|
||||
PlatformDependent.freeDirectBuffer(buffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int arrayOffset() {
|
||||
return this.buffer.arrayOffset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer reclaimRead() {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte get() {
|
||||
return this.buffer.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte get(int index) {
|
||||
return this.buffer.get(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt() {
|
||||
return this.buffer.getInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong() {
|
||||
return this.buffer.getLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getShort() {
|
||||
return this.buffer.getShort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat() {
|
||||
return this.buffer.getFloat();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble() {
|
||||
return this.buffer.getDouble();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer get(byte[] target, int offset, int length) {
|
||||
this.buffer.get(target, offset, length);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer get(byte[] target) {
|
||||
this.buffer.get(target);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer get(WritableBuffer target) {
|
||||
int start = target.position();
|
||||
if (this.buffer.hasArray()) {
|
||||
target.put(this.buffer.array(), this.buffer.arrayOffset() + this.buffer.position(), this.buffer.remaining());
|
||||
} else {
|
||||
target.put(this.buffer);
|
||||
}
|
||||
|
||||
int written = target.position() - start;
|
||||
this.buffer.position(this.buffer.position() + written);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer slice() {
|
||||
return new AmqpReadableBuffer(this.buffer.slice());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer flip() {
|
||||
this.buffer.flip();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer limit(int limit) {
|
||||
this.buffer.limit(limit);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int limit() {
|
||||
return this.buffer.limit();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer position(int position) {
|
||||
this.buffer.position(position);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int position() {
|
||||
return this.buffer.position();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer mark() {
|
||||
this.buffer.mark();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer reset() {
|
||||
this.buffer.reset();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer rewind() {
|
||||
this.buffer.rewind();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer clear() {
|
||||
this.buffer.clear();
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int remaining() {
|
||||
return this.buffer.remaining();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasRemaining() {
|
||||
return this.buffer.hasRemaining();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableBuffer duplicate() {
|
||||
return new AmqpReadableBuffer(this.buffer.duplicate());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer byteBuffer() {
|
||||
return this.buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String readUTF8() throws CharacterCodingException {
|
||||
ByteBuf wrappedNetty = Unpooled.wrappedBuffer(buffer);
|
||||
return wrappedNetty.toString(StandardCharsets.UTF_8);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String readString(CharsetDecoder decoder) throws CharacterCodingException {
|
||||
ByteBuf wrappedNetty = Unpooled.wrappedBuffer(buffer);
|
||||
return wrappedNetty.toString(StandardCharsets.UTF_8);
|
||||
}
|
||||
}
|
@ -65,6 +65,11 @@ public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage,
|
||||
|
||||
private final Map<SimpleString, RoutingType> prefixes = new HashMap<>();
|
||||
|
||||
/** minLargeMessageSize determines when a message should be considered as large.
|
||||
* minLargeMessageSize = -1 basically disables large message control over AMQP.
|
||||
*/
|
||||
private int amqpMinLargeMessageSize = 100 * 1024;
|
||||
|
||||
private int amqpCredits = AmqpSupport.AMQP_CREDITS_DEFAULT;
|
||||
|
||||
private int amqpLowCredits = AmqpSupport.AMQP_LOW_CREDITS_DEFAULT;
|
||||
@ -112,6 +117,15 @@ public class ProtonProtocolManager extends AbstractProtocolManager<AMQPMessage,
|
||||
|
||||
}
|
||||
|
||||
public int getAmqpMinLargeMessageSize() {
|
||||
return amqpMinLargeMessageSize;
|
||||
}
|
||||
|
||||
public ProtonProtocolManager setAmqpMinLargeMessageSize(int amqpMinLargeMessageSize) {
|
||||
this.amqpMinLargeMessageSize = amqpMinLargeMessageSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean isAmqpDuplicateDetection() {
|
||||
return amqpDuplicateDetection;
|
||||
}
|
||||
|
@ -21,7 +21,6 @@ import java.util.Map;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.BaseInterceptor;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
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;
|
||||
@ -40,9 +39,9 @@ public class ProtonProtocolManagerFactory extends AbstractProtocolManagerFactory
|
||||
private static String[] SUPPORTED_PROTOCOLS = {AMQP_PROTOCOL_NAME};
|
||||
|
||||
@Override
|
||||
public Persister<Message, CoreMessageObjectPools>[] getPersister() {
|
||||
public Persister<Message>[] getPersister() {
|
||||
|
||||
Persister[] persisters = new Persister[]{AMQPMessagePersister.getInstance(), AMQPMessagePersisterV2.getInstance()};
|
||||
Persister[] persisters = new Persister[]{AMQPMessagePersister.getInstance(), AMQPMessagePersisterV2.getInstance(), AMQPLargeMessagePersister.getInstance()};
|
||||
return persisters;
|
||||
}
|
||||
|
||||
|
@ -17,7 +17,8 @@
|
||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.MessageConverter;
|
||||
|
||||
@ -34,8 +35,8 @@ public class AMQPConverter implements MessageConverter<AMQPMessage> {
|
||||
}
|
||||
|
||||
@Override
|
||||
public AMQPMessage fromCore(ICoreMessage coreMessage) throws Exception {
|
||||
return CoreAmqpConverter.fromCore(coreMessage);
|
||||
public AMQPMessage fromCore(ICoreMessage coreMessage, StorageManager storageManager) throws Exception {
|
||||
return CoreAmqpConverter.fromCore(coreMessage, storageManager);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -36,7 +36,7 @@ import javax.jms.TemporaryTopic;
|
||||
import javax.jms.Topic;
|
||||
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQQueue;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQTemporaryQueue;
|
||||
|
@ -64,7 +64,7 @@ import javax.jms.JMSException;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQPropertyConversionException;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.jms.ServerJMSMessage;
|
||||
|
@ -68,7 +68,9 @@ import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPStandardMessage;
|
||||
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;
|
||||
@ -107,22 +109,22 @@ public class CoreAmqpConverter {
|
||||
|
||||
private static Logger logger = Logger.getLogger(CoreAmqpConverter.class);
|
||||
|
||||
public static AMQPMessage checkAMQP(Message message) throws Exception {
|
||||
public static AMQPMessage checkAMQP(Message message, StorageManager storageManager) throws Exception {
|
||||
if (message instanceof AMQPMessage) {
|
||||
return (AMQPMessage)message;
|
||||
} else {
|
||||
// It will first convert to Core, then to AMQP
|
||||
return fromCore(message.toCore());
|
||||
return fromCore(message.toCore(), storageManager);
|
||||
}
|
||||
}
|
||||
|
||||
public static AMQPMessage fromCore(ICoreMessage coreMessage) throws Exception {
|
||||
public static AMQPMessage fromCore(ICoreMessage coreMessage, StorageManager storageManager) throws Exception {
|
||||
if (coreMessage == null) {
|
||||
return null;
|
||||
}
|
||||
if (coreMessage.isServerMessage() && coreMessage.isLargeMessage() && coreMessage.getType() == EMBEDDED_TYPE) {
|
||||
//large AMQP messages received across cluster nodes
|
||||
final Message message = EmbedMessageUtil.extractEmbedded(coreMessage);
|
||||
final Message message = EmbedMessageUtil.extractEmbedded(coreMessage, storageManager);
|
||||
if (message instanceof AMQPMessage) {
|
||||
return (AMQPMessage) message;
|
||||
}
|
||||
@ -363,7 +365,7 @@ public class CoreAmqpConverter {
|
||||
byte[] data = new byte[buffer.writerIndex()];
|
||||
buffer.readBytes(data);
|
||||
|
||||
AMQPMessage amqpMessage = new AMQPMessage(messageFormat, data, null);
|
||||
AMQPMessage amqpMessage = new AMQPStandardMessage(messageFormat, data, null);
|
||||
amqpMessage.setMessageID(message.getInnerMessage().getMessageID());
|
||||
amqpMessage.setReplyTo(coreMessage.getReplyTo());
|
||||
return amqpMessage;
|
||||
|
@ -188,6 +188,9 @@ public class AMQPConnectionContext extends ProtonInitializable implements EventH
|
||||
return false;
|
||||
}
|
||||
|
||||
public void instantFlush() {
|
||||
handler.instantFlush();
|
||||
}
|
||||
public void flush() {
|
||||
handler.flush();
|
||||
}
|
||||
|
@ -25,12 +25,15 @@ import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQSecurityException;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.nullpm.NullStorageManager;
|
||||
import org.apache.activemq.artemis.core.security.CheckType;
|
||||
import org.apache.activemq.artemis.core.security.SecurityAuth;
|
||||
import org.apache.activemq.artemis.core.server.RoutingContext;
|
||||
import org.apache.activemq.artemis.core.server.impl.AddressInfo;
|
||||
import org.apache.activemq.artemis.core.server.impl.RoutingContextImpl;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessage;
|
||||
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.ActiveMQAMQPInternalErrorException;
|
||||
@ -121,6 +124,8 @@ public class ProtonServerReceiverContext extends ProtonInitializable implements
|
||||
// Used by the broker to decide when to refresh clients credit. This is not used when client requests credit.
|
||||
private final int minCreditRefresh;
|
||||
|
||||
private final int minLargeMessageSize;
|
||||
|
||||
public ProtonServerReceiverContext(AMQPSessionCallback sessionSPI,
|
||||
AMQPConnectionContext connection,
|
||||
AMQPSessionContext protonSession,
|
||||
@ -134,6 +139,7 @@ public class ProtonServerReceiverContext extends ProtonInitializable implements
|
||||
this.minCreditRefresh = connection.getAmqpLowCredits();
|
||||
this.creditRunnable = createCreditRunnable(amqpCredits, minCreditRefresh, receiver, connection).setRan();
|
||||
useModified = this.connection.getProtocolManager().isUseModifiedForTransientDeliveryErrors();
|
||||
this.minLargeMessageSize = connection.getProtocolManager().getAmqpMinLargeMessageSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -264,6 +270,8 @@ public class ProtonServerReceiverContext extends ProtonInitializable implements
|
||||
return defaultRoutingType;
|
||||
}
|
||||
|
||||
volatile AMQPLargeMessage currentLargeMessage;
|
||||
|
||||
/*
|
||||
* called when Proton receives a message to be delivered via a Delivery.
|
||||
*
|
||||
@ -278,39 +286,78 @@ public class ProtonServerReceiverContext extends ProtonInitializable implements
|
||||
return;
|
||||
}
|
||||
|
||||
if (delivery.isAborted()) {
|
||||
// Aborting implicitly remotely settles, so advance
|
||||
// receiver to the next delivery and settle locally.
|
||||
receiver.advance();
|
||||
delivery.settle();
|
||||
try {
|
||||
if (delivery.isAborted()) {
|
||||
// Aborting implicitly remotely settles, so advance
|
||||
// receiver to the next delivery and settle locally.
|
||||
receiver.advance();
|
||||
delivery.settle();
|
||||
|
||||
// Replenish the credit if not doing a drain
|
||||
if (!receiver.getDrain()) {
|
||||
receiver.flow(1);
|
||||
// Replenish the credit if not doing a drain
|
||||
if (!receiver.getDrain()) {
|
||||
receiver.flow(1);
|
||||
}
|
||||
|
||||
return;
|
||||
} else if (delivery.isPartial()) {
|
||||
if (sessionSPI.getStorageManager() instanceof NullStorageManager) {
|
||||
// if we are dealing with the NullStorageManager we should just make it a regular message anyways
|
||||
return;
|
||||
}
|
||||
|
||||
if (currentLargeMessage == null) {
|
||||
// minLargeMessageSize < 0 means no large message treatment, make it disabled
|
||||
if (minLargeMessageSize > 0 && delivery.available() >= minLargeMessageSize) {
|
||||
initializeCurrentLargeMessage(delivery, receiver);
|
||||
}
|
||||
} else {
|
||||
currentLargeMessage.addBytes(receiver.recv());
|
||||
}
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
return;
|
||||
} else if (delivery.isPartial()) {
|
||||
return;
|
||||
AMQPMessage message;
|
||||
|
||||
// this is treating the case where the frameSize > minLargeMessage and the message is still large enough
|
||||
if (!(sessionSPI.getStorageManager() instanceof NullStorageManager) && currentLargeMessage == null && minLargeMessageSize > 0 && delivery.available() >= minLargeMessageSize) {
|
||||
initializeCurrentLargeMessage(delivery, receiver);
|
||||
}
|
||||
|
||||
if (currentLargeMessage != null) {
|
||||
currentLargeMessage.addBytes(receiver.recv());
|
||||
receiver.advance();
|
||||
currentLargeMessage.finishParse();
|
||||
message = currentLargeMessage;
|
||||
currentLargeMessage = null;
|
||||
} else {
|
||||
ReadableBuffer data = receiver.recv();
|
||||
receiver.advance();
|
||||
message = sessionSPI.createStandardMessage(delivery, data);
|
||||
}
|
||||
|
||||
Transaction tx = null;
|
||||
if (delivery.getRemoteState() instanceof TransactionalState) {
|
||||
TransactionalState txState = (TransactionalState) delivery.getRemoteState();
|
||||
tx = this.sessionSPI.getTransaction(txState.getTxnId(), false);
|
||||
}
|
||||
|
||||
actualDelivery(message, delivery, receiver, tx);
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQAMQPInternalErrorException(e.getMessage(), e);
|
||||
}
|
||||
|
||||
ReadableBuffer data = receiver.recv();
|
||||
receiver.advance();
|
||||
Transaction tx = null;
|
||||
|
||||
if (delivery.getRemoteState() instanceof TransactionalState) {
|
||||
TransactionalState txState = (TransactionalState) delivery.getRemoteState();
|
||||
tx = this.sessionSPI.getTransaction(txState.getTxnId(), false);
|
||||
}
|
||||
|
||||
final Transaction txUsed = tx;
|
||||
|
||||
actualDelivery(delivery, receiver, data, txUsed);
|
||||
}
|
||||
|
||||
private void actualDelivery(Delivery delivery, Receiver receiver, ReadableBuffer data, Transaction tx) {
|
||||
private void initializeCurrentLargeMessage(Delivery delivery, Receiver receiver) throws Exception {
|
||||
long id = sessionSPI.getStorageManager().generateID();
|
||||
currentLargeMessage = new AMQPLargeMessage(id, delivery.getMessageFormat(), null, sessionSPI.getCoreMessageObjectPools(), sessionSPI.getStorageManager());
|
||||
currentLargeMessage.addBytes(receiver.recv());
|
||||
}
|
||||
|
||||
private void actualDelivery(AMQPMessage message, Delivery delivery, Receiver receiver, Transaction tx) {
|
||||
try {
|
||||
sessionSPI.serverSend(this, tx, receiver, delivery, address, delivery.getMessageFormat(), data, routingContext);
|
||||
sessionSPI.serverSend(this, tx, receiver, delivery, address, routingContext, message);
|
||||
} catch (Exception e) {
|
||||
log.warn(e.getMessage(), e);
|
||||
DeliveryState deliveryState = determineDeliveryState(((Source) receiver.getSource()),
|
||||
|
@ -16,6 +16,7 @@
|
||||
*/
|
||||
package org.apache.activemq.artemis.protocol.amqp.proton;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
@ -29,6 +30,7 @@ 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.io.IOCallback;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.persistence.OperationContext;
|
||||
import org.apache.activemq.artemis.core.server.AddressQueryResult;
|
||||
import org.apache.activemq.artemis.core.server.Consumer;
|
||||
@ -37,6 +39,7 @@ import org.apache.activemq.artemis.core.server.QueueQueryResult;
|
||||
import org.apache.activemq.artemis.core.server.ServerConsumer;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerConsumerImpl;
|
||||
import org.apache.activemq.artemis.jms.client.ActiveMQDestination;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPLargeMessage;
|
||||
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.broker.ActiveMQProtonRemotingConnection;
|
||||
@ -110,6 +113,12 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
||||
private SimpleString tempQueueName;
|
||||
private final AtomicBoolean draining = new AtomicBoolean(false);
|
||||
|
||||
// once a large message is accepted, we shouldn't accept any further messages
|
||||
// as large message could be interrupted due to flow control and resumed at the same message
|
||||
volatile boolean hasLarge = false;
|
||||
volatile LargeMessageDeliveryContext pendingLargeMessage = null;
|
||||
|
||||
|
||||
private int credits = 0;
|
||||
|
||||
private AtomicInteger pending = new AtomicInteger(0);
|
||||
@ -170,6 +179,11 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
||||
}
|
||||
|
||||
public boolean hasCredits() {
|
||||
if (hasLarge) {
|
||||
// we will resume accepting once the large message is finished
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!connection.flowControl(onflowControlReady)) {
|
||||
return false;
|
||||
}
|
||||
@ -764,9 +778,13 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
||||
credits--;
|
||||
}
|
||||
|
||||
if (messageReference.getMessage() instanceof AMQPLargeMessage) {
|
||||
hasLarge = true;
|
||||
}
|
||||
|
||||
if (messageReference instanceof Runnable && consumer.allowReferenceCallback()) {
|
||||
messageReference.onDelivery(executeDelivery);
|
||||
connection.runNow((Runnable)messageReference);
|
||||
connection.runNow((Runnable) messageReference);
|
||||
} else {
|
||||
connection.runNow(() -> executeDelivery(messageReference));
|
||||
}
|
||||
@ -785,39 +803,78 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
||||
log.debug("Not delivering message " + messageReference + " as the sender is closed and credits were available, if you see too many of these it means clients are issuing credits and closing the connection with pending credits a lot of times");
|
||||
return;
|
||||
}
|
||||
AMQPMessage message = CoreAmqpConverter.checkAMQP(messageReference.getMessage());
|
||||
AMQPMessage message = CoreAmqpConverter.checkAMQP(messageReference.getMessage(), sessionSPI.getStorageManager());
|
||||
|
||||
if (sessionSPI.invokeOutgoing(message, (ActiveMQProtonRemotingConnection) sessionSPI.getTransportConnection().getProtocolConnection()) != null) {
|
||||
return;
|
||||
}
|
||||
if (message instanceof AMQPLargeMessage) {
|
||||
deliverLarge(messageReference, (AMQPLargeMessage) message);
|
||||
} else {
|
||||
deliverStandard(messageReference, message);
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
log.warn(e.getMessage(), e);
|
||||
brokerConsumer.errorProcessing(e, messageReference);
|
||||
}
|
||||
}
|
||||
|
||||
private class LargeMessageDeliveryContext {
|
||||
|
||||
LargeMessageDeliveryContext(MessageReference reference, AMQPLargeMessage message, Delivery delivery) {
|
||||
this.position = 0L;
|
||||
this.reference = reference;
|
||||
this.message = message;
|
||||
this.delivery = delivery;
|
||||
}
|
||||
|
||||
long position;
|
||||
final MessageReference reference;
|
||||
final AMQPLargeMessage message;
|
||||
final Delivery delivery;
|
||||
|
||||
void resume() {
|
||||
connection.runNow(this::deliver);
|
||||
}
|
||||
private static final int BUFFER_LENGTH = 1024;
|
||||
|
||||
void deliver() {
|
||||
// Let the Message decide how to present the message bytes
|
||||
ReadableBuffer sendBuffer = message.getSendBuffer(messageReference.getDeliveryCount());
|
||||
// we only need a tag if we are going to settle later
|
||||
byte[] tag = preSettle ? new byte[0] : protonSession.getTag();
|
||||
|
||||
boolean releaseRequired = sendBuffer instanceof NettyReadable;
|
||||
final Delivery delivery;
|
||||
delivery = sender.delivery(tag, 0, tag.length);
|
||||
delivery.setMessageFormat((int) message.getMessageFormat());
|
||||
delivery.setContext(messageReference);
|
||||
|
||||
LargeBodyReader context = message.getLargeBodyReader();
|
||||
try {
|
||||
|
||||
if (releaseRequired) {
|
||||
sender.send(sendBuffer);
|
||||
// Above send copied, so release now if needed
|
||||
releaseRequired = false;
|
||||
((NettyReadable) sendBuffer).getByteBuf().release();
|
||||
} else {
|
||||
// Don't have pooled content, no need to release or copy.
|
||||
sender.sendNoCopy(sendBuffer);
|
||||
context.open();
|
||||
try {
|
||||
context.position(position);
|
||||
long bodySize = context.getSize();
|
||||
|
||||
// TODO: it would be nice to use pooled buffer here,
|
||||
// however I would need a version of ReadableBuffer for Netty
|
||||
ByteBuffer buf = ByteBuffer.allocate(BUFFER_LENGTH);
|
||||
|
||||
for (; position < bodySize; ) {
|
||||
if (!connection.flowControl(this::resume)) {
|
||||
context.close();
|
||||
return;
|
||||
}
|
||||
buf.clear();
|
||||
int size = context.readInto(buf);
|
||||
|
||||
sender.send(buf.array(), 0, size);
|
||||
|
||||
connection.instantFlush();
|
||||
|
||||
position += size;
|
||||
}
|
||||
} finally {
|
||||
context.close();
|
||||
}
|
||||
|
||||
if (preSettle) {
|
||||
// Presettled means the client implicitly accepts any delivery we send it.
|
||||
try {
|
||||
sessionSPI.ack(null, brokerConsumer, messageReference.getMessage());
|
||||
sessionSPI.ack(null, brokerConsumer, reference.getMessage());
|
||||
} catch (Exception e) {
|
||||
log.debug(e.getMessage(), e);
|
||||
}
|
||||
@ -827,17 +884,84 @@ public class ProtonServerSenderContext extends ProtonInitializable implements Pr
|
||||
}
|
||||
|
||||
connection.flush();
|
||||
} finally {
|
||||
|
||||
synchronized (creditsLock) {
|
||||
pending.decrementAndGet();
|
||||
}
|
||||
if (releaseRequired) {
|
||||
((NettyReadable) sendBuffer).getByteBuf().release();
|
||||
}
|
||||
|
||||
finishLargeMessage();
|
||||
} catch (Exception e) {
|
||||
log.warn(e.getMessage(), e);
|
||||
brokerConsumer.errorProcessing(e, reference);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void finishLargeMessage() {
|
||||
pendingLargeMessage = null;
|
||||
hasLarge = false;
|
||||
brokerConsumer.promptDelivery();
|
||||
}
|
||||
|
||||
private void deliverLarge(MessageReference messageReference, AMQPLargeMessage message) {
|
||||
|
||||
// we only need a tag if we are going to settle later
|
||||
byte[] tag = preSettle ? new byte[0] : protonSession.getTag();
|
||||
|
||||
final Delivery delivery;
|
||||
delivery = sender.delivery(tag, 0, tag.length);
|
||||
delivery.setMessageFormat((int) message.getMessageFormat());
|
||||
delivery.setContext(messageReference);
|
||||
|
||||
pendingLargeMessage = new LargeMessageDeliveryContext(messageReference, message, delivery);
|
||||
pendingLargeMessage.deliver();
|
||||
|
||||
}
|
||||
|
||||
private void deliverStandard(MessageReference messageReference, AMQPMessage message) {
|
||||
// Let the Message decide how to present the message bytes
|
||||
ReadableBuffer sendBuffer = message.getSendBuffer(messageReference.getDeliveryCount());
|
||||
// we only need a tag if we are going to settle later
|
||||
byte[] tag = preSettle ? new byte[0] : protonSession.getTag();
|
||||
|
||||
boolean releaseRequired = sendBuffer instanceof NettyReadable;
|
||||
final Delivery delivery;
|
||||
delivery = sender.delivery(tag, 0, tag.length);
|
||||
delivery.setMessageFormat((int) message.getMessageFormat());
|
||||
delivery.setContext(messageReference);
|
||||
|
||||
try {
|
||||
|
||||
if (releaseRequired) {
|
||||
sender.send(sendBuffer);
|
||||
// Above send copied, so release now if needed
|
||||
releaseRequired = false;
|
||||
((NettyReadable) sendBuffer).getByteBuf().release();
|
||||
} else {
|
||||
// Don't have pooled content, no need to release or copy.
|
||||
sender.sendNoCopy(sendBuffer);
|
||||
}
|
||||
|
||||
if (preSettle) {
|
||||
// Presettled means the client implicitly accepts any delivery we send it.
|
||||
try {
|
||||
sessionSPI.ack(null, brokerConsumer, messageReference.getMessage());
|
||||
} catch (Exception e) {
|
||||
log.debug(e.getMessage(), e);
|
||||
}
|
||||
delivery.settle();
|
||||
} else {
|
||||
sender.advance();
|
||||
}
|
||||
|
||||
connection.flush();
|
||||
} finally {
|
||||
synchronized (creditsLock) {
|
||||
pending.decrementAndGet();
|
||||
}
|
||||
if (releaseRequired) {
|
||||
((NettyReadable) sendBuffer).getByteBuf().release();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.warn(e.getMessage(), e);
|
||||
brokerConsumer.errorProcessing(e, messageReference);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -88,6 +88,8 @@ public class ProtonHandler extends ProtonInitializable implements SaslListener {
|
||||
|
||||
boolean scheduledFlush = false;
|
||||
|
||||
boolean flushInstantly = false;
|
||||
|
||||
public ProtonHandler(EventLoop workerExecutor, ArtemisExecutor poolExecutor, boolean isServer) {
|
||||
this.workerExecutor = workerExecutor;
|
||||
this.poolExecutor = poolExecutor;
|
||||
@ -174,12 +176,29 @@ public class ProtonHandler extends ProtonInitializable implements SaslListener {
|
||||
|
||||
|
||||
|
||||
/** When processing large messages, we require to flush bytes every processing */
|
||||
public void instantFlush() {
|
||||
this.flushInstantly = true;
|
||||
// This will perform event handling, and at some point the flushBytes will be called
|
||||
this.flush();
|
||||
}
|
||||
|
||||
public void flushBytes() {
|
||||
requireHandler();
|
||||
|
||||
if (!scheduledFlush) {
|
||||
scheduledFlush = true;
|
||||
workerExecutor.execute(this::actualFlush);
|
||||
if (flushInstantly) {
|
||||
flushInstantly = false;
|
||||
scheduledFlush = false;
|
||||
actualFlush();
|
||||
} else {
|
||||
// Under regular circunstances, it would be too costly to flush every time,
|
||||
// so we flush only once at the end of processing.
|
||||
|
||||
// this decision was made after extensive performance testing.
|
||||
if (!scheduledFlush) {
|
||||
scheduledFlush = true;
|
||||
workerExecutor.execute(this::actualFlush);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -30,6 +30,7 @@ import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ResetLimitWrappedActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPStandardMessage;
|
||||
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;
|
||||
@ -168,7 +169,7 @@ public class TestConversions extends Assert {
|
||||
assertEquals(1, mapMessage.getInt("someint"));
|
||||
assertEquals("value", mapMessage.getString("somestr"));
|
||||
|
||||
AMQPMessage newAMQP = CoreAmqpConverter.fromCore(mapMessage.getInnerMessage());
|
||||
AMQPMessage newAMQP = CoreAmqpConverter.fromCore(mapMessage.getInnerMessage(), null);
|
||||
assertNotNull(newAMQP.getBody());
|
||||
}
|
||||
|
||||
@ -286,7 +287,7 @@ public class TestConversions extends Assert {
|
||||
assertTrue(mapMessage.propertyExists(JMS_AMQP_ENCODED_MESSAGE_ANNOTATION_PREFIX + annotationName));
|
||||
assertArrayEquals(encodedEmbeddedMap, (byte[]) mapMessage.getObjectProperty(JMS_AMQP_ENCODED_MESSAGE_ANNOTATION_PREFIX + annotationName));
|
||||
|
||||
AMQPMessage newAMQP = CoreAmqpConverter.fromCore(mapMessage.getInnerMessage());
|
||||
AMQPMessage newAMQP = CoreAmqpConverter.fromCore(mapMessage.getInnerMessage(), null);
|
||||
assertNotNull(newAMQP.getBody());
|
||||
assertNotNull(newAMQP.getMessageAnnotations());
|
||||
assertNotNull(newAMQP.getMessageAnnotations().getValue());
|
||||
@ -338,7 +339,7 @@ public class TestConversions extends Assert {
|
||||
assertTrue(mapMessage.propertyExists(JMS_AMQP_ENCODED_FOOTER_PREFIX + footerName));
|
||||
assertArrayEquals(encodedEmbeddedMap, (byte[]) mapMessage.getObjectProperty(JMS_AMQP_ENCODED_FOOTER_PREFIX + footerName));
|
||||
|
||||
AMQPMessage newAMQP = CoreAmqpConverter.fromCore(mapMessage.getInnerMessage());
|
||||
AMQPMessage newAMQP = CoreAmqpConverter.fromCore(mapMessage.getInnerMessage(), null);
|
||||
assertNotNull(newAMQP.getBody());
|
||||
assertNotNull(newAMQP.getFooter());
|
||||
assertNotNull(newAMQP.getFooter().getValue());
|
||||
@ -368,7 +369,7 @@ public class TestConversions extends Assert {
|
||||
serverMessage.setObjectProperty(JMS_AMQP_ENCODED_DELIVERY_ANNOTATION_PREFIX + annotationName, encodedEmbeddedMap);
|
||||
serverMessage.encode();
|
||||
|
||||
AMQPMessage newAMQP = CoreAmqpConverter.fromCore(serverMessage.getInnerMessage());
|
||||
AMQPMessage newAMQP = CoreAmqpConverter.fromCore(serverMessage.getInnerMessage(), null);
|
||||
assertNull(newAMQP.getBody());
|
||||
assertNotNull(newAMQP.getDeliveryAnnotations());
|
||||
assertNotNull(newAMQP.getDeliveryAnnotations().getValue());
|
||||
@ -520,7 +521,7 @@ public class TestConversions extends Assert {
|
||||
|
||||
NettyReadable readable = new NettyReadable(encoded.getByteBuf());
|
||||
|
||||
return new AMQPMessage(AMQPMessage.DEFAULT_MESSAGE_FORMAT, readable, null, null);
|
||||
return new AMQPStandardMessage(AMQPMessage.DEFAULT_MESSAGE_FORMAT, readable, null, null);
|
||||
}
|
||||
|
||||
private ServerJMSMessage createMessage() {
|
||||
|
@ -38,6 +38,7 @@ import javax.jms.Topic;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPStandardMessage;
|
||||
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;
|
||||
@ -80,7 +81,7 @@ public class JMSMappingInboundTransformerTest {
|
||||
MessageImpl message = (MessageImpl) Message.Factory.create();
|
||||
message.setContentType(AMQPMessageSupport.OCTET_STREAM_CONTENT_TYPE);
|
||||
|
||||
AMQPMessage messageEncode = encodeAndCreateAMQPMessage(message);
|
||||
AMQPStandardMessage messageEncode = encodeAndCreateAMQPMessage(message);
|
||||
|
||||
ICoreMessage coreMessage = messageEncode.toCore();
|
||||
|
||||
@ -135,7 +136,7 @@ public class JMSMappingInboundTransformerTest {
|
||||
message.setBody(new Data(binary));
|
||||
message.setContentType(AMQPMessageSupport.OCTET_STREAM_CONTENT_TYPE);
|
||||
|
||||
AMQPMessage amqp = encodeAndCreateAMQPMessage(message);
|
||||
AMQPStandardMessage amqp = encodeAndCreateAMQPMessage(message);
|
||||
javax.jms.Message jmsMessage = ServerJMSMessage.wrapCoreMessage(amqp.toCore());
|
||||
|
||||
assertNotNull("Message should not be null", jmsMessage);
|
||||
@ -570,12 +571,12 @@ public class JMSMappingInboundTransformerTest {
|
||||
assertTrue("Expected TextMessage", jmsMessage instanceof TextMessage);
|
||||
}
|
||||
|
||||
private AMQPMessage encodeAndCreateAMQPMessage(MessageImpl message) {
|
||||
private AMQPStandardMessage encodeAndCreateAMQPMessage(MessageImpl message) {
|
||||
NettyWritable encoded = new NettyWritable(Unpooled.buffer(1024));
|
||||
message.encode(encoded);
|
||||
|
||||
NettyReadable readable = new NettyReadable(encoded.getByteBuf());
|
||||
|
||||
return new AMQPMessage(AMQPMessage.DEFAULT_MESSAGE_FORMAT, readable, null, null);
|
||||
return new AMQPStandardMessage(AMQPMessage.DEFAULT_MESSAGE_FORMAT, readable, null, null);
|
||||
}
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
ServerJMSMessage outbound = createMessage();
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNull(amqp.getBody());
|
||||
}
|
||||
@ -95,7 +95,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_NULL);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNull(amqp.getBody());
|
||||
}
|
||||
@ -109,7 +109,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.writeBytes(expectedPayload);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
@ -128,7 +128,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -144,7 +144,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.writeBytes(expectedPayload);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -164,7 +164,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
ServerJMSMapMessage outbound = createMapMessage();
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -179,7 +179,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setBytes("bytes", byteArray);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -201,7 +201,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setBoolean("property-3", true);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -223,7 +223,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.writeString("test");
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -244,7 +244,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.writeString("test");
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -265,7 +265,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.writeString("test");
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpSequence);
|
||||
@ -284,7 +284,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage();
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
@ -297,7 +297,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_UNKNOWN);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
@ -309,7 +309,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
ServerJMSObjectMessage outbound = createObjectMessage(TEST_OBJECT_VALUE);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
@ -326,7 +326,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_UNKNOWN);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
@ -343,7 +343,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -361,7 +361,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_VALUE_BINARY);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -376,7 +376,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
ServerJMSTextMessage outbound = createTextMessage();
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -389,7 +389,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
ServerJMSTextMessage outbound = createTextMessage(contentString);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -402,7 +402,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
ServerJMSTextMessage outbound = createTextMessage(contentString);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -415,7 +415,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
ServerJMSTextMessage outbound = createTextMessage(contentString);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof AmqpValue);
|
||||
@ -432,7 +432,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_DATA);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
@ -450,7 +450,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
outbound.setShortProperty(JMS_AMQP_ORIGINAL_ENCODING, AMQP_DATA);
|
||||
outbound.encode();
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(outbound.getInnerMessage(), null);
|
||||
|
||||
assertNotNull(amqp.getBody());
|
||||
assertTrue(amqp.getBody() instanceof Data);
|
||||
@ -478,7 +478,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
textMessage.setText("myTextMessageContent");
|
||||
textMessage.setJMSDestination(jmsDestination);
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(textMessage.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(textMessage.getInnerMessage(), null);
|
||||
|
||||
MessageAnnotations ma = amqp.getMessageAnnotations();
|
||||
Map<Symbol, Object> maMap = ma == null ? null : ma.getValue();
|
||||
@ -511,7 +511,7 @@ public class JMSMappingOutboundTransformerTest {
|
||||
textMessage.setText("myTextMessageContent");
|
||||
textMessage.setJMSReplyTo(jmsReplyTo);
|
||||
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(textMessage.getInnerMessage());
|
||||
AMQPMessage amqp = AMQPConverter.getInstance().fromCore(textMessage.getInnerMessage(), null);
|
||||
|
||||
MessageAnnotations ma = amqp.getMessageAnnotations();
|
||||
Map<Symbol, Object> maMap = ma == null ? null : ma.getValue();
|
||||
|
@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPStandardMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPConverter;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyReadable;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
@ -62,12 +63,12 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
public void testBodyOnlyMessage() throws Exception {
|
||||
MessageImpl message = (MessageImpl) Proton.message();
|
||||
message.setBody(new AmqpValue("String payload for AMQP message conversion performance testing."));
|
||||
AMQPMessage encoded = encodeAndCreateAMQPMessage(message);
|
||||
AMQPStandardMessage encoded = encodeAndCreateAMQPMessage(message);
|
||||
|
||||
// Warm up
|
||||
for (int i = 0; i < WARM_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
|
||||
long totalDuration = 0;
|
||||
@ -75,7 +76,7 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
long startTime = System.nanoTime();
|
||||
for (int i = 0; i < PROFILE_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
totalDuration += System.nanoTime() - startTime;
|
||||
|
||||
@ -92,12 +93,12 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
message.setContentType("text/plain");
|
||||
message.setBody(new AmqpValue("String payload for AMQP message conversion performance testing."));
|
||||
|
||||
AMQPMessage encoded = encodeAndCreateAMQPMessage(message);
|
||||
AMQPStandardMessage encoded = encodeAndCreateAMQPMessage(message);
|
||||
|
||||
// Warm up
|
||||
for (int i = 0; i < WARM_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
|
||||
long totalDuration = 0;
|
||||
@ -105,7 +106,7 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
long startTime = System.nanoTime();
|
||||
for (int i = 0; i < PROFILE_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
totalDuration += System.nanoTime() - startTime;
|
||||
|
||||
@ -114,12 +115,12 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
|
||||
@Test
|
||||
public void testTypicalQpidJMSMessage() throws Exception {
|
||||
AMQPMessage encoded = encodeAndCreateAMQPMessage(createTypicalQpidJMSMessage());
|
||||
AMQPStandardMessage encoded = encodeAndCreateAMQPMessage(createTypicalQpidJMSMessage());
|
||||
|
||||
// Warm up
|
||||
for (int i = 0; i < WARM_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
|
||||
long totalDuration = 0;
|
||||
@ -127,7 +128,7 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
long startTime = System.nanoTime();
|
||||
for (int i = 0; i < PROFILE_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
totalDuration += System.nanoTime() - startTime;
|
||||
|
||||
@ -137,12 +138,12 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
@Test
|
||||
public void testComplexQpidJMSMessage() throws Exception {
|
||||
|
||||
AMQPMessage encoded = encodeAndCreateAMQPMessage(createComplexQpidJMSMessage());
|
||||
AMQPStandardMessage encoded = encodeAndCreateAMQPMessage(createComplexQpidJMSMessage());
|
||||
|
||||
// Warm up
|
||||
for (int i = 0; i < WARM_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
|
||||
long totalDuration = 0;
|
||||
@ -150,7 +151,7 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
long startTime = System.nanoTime();
|
||||
for (int i = 0; i < PROFILE_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
totalDuration += System.nanoTime() - startTime;
|
||||
|
||||
@ -160,12 +161,12 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
@Test
|
||||
public void testTypicalQpidJMSMessageInBoundOnly() throws Exception {
|
||||
|
||||
AMQPMessage encoded = encodeAndCreateAMQPMessage(createTypicalQpidJMSMessage());
|
||||
AMQPStandardMessage encoded = encodeAndCreateAMQPMessage(createTypicalQpidJMSMessage());
|
||||
|
||||
// Warm up
|
||||
for (int i = 0; i < WARM_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
|
||||
long totalDuration = 0;
|
||||
@ -173,7 +174,7 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
long startTime = System.nanoTime();
|
||||
for (int i = 0; i < PROFILE_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
|
||||
totalDuration += System.nanoTime() - startTime;
|
||||
@ -183,12 +184,12 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
|
||||
@Test
|
||||
public void testTypicalQpidJMSMessageOutBoundOnly() throws Exception {
|
||||
AMQPMessage encoded = encodeAndCreateAMQPMessage(createTypicalQpidJMSMessage());
|
||||
AMQPStandardMessage encoded = encodeAndCreateAMQPMessage(createTypicalQpidJMSMessage());
|
||||
|
||||
// Warm up
|
||||
for (int i = 0; i < WARM_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
|
||||
long totalDuration = 0;
|
||||
@ -196,7 +197,7 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
long startTime = System.nanoTime();
|
||||
for (int i = 0; i < PROFILE_CYCLES; ++i) {
|
||||
ICoreMessage intermediate = encoded.toCore();
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate));
|
||||
encode(AMQPConverter.getInstance().fromCore(intermediate,null));
|
||||
}
|
||||
|
||||
totalDuration += System.nanoTime() - startTime;
|
||||
@ -272,13 +273,13 @@ public class JMSTransformationSpeedComparisonTest {
|
||||
return message;
|
||||
}
|
||||
|
||||
private AMQPMessage encodeAndCreateAMQPMessage(MessageImpl message) {
|
||||
private AMQPStandardMessage encodeAndCreateAMQPMessage(MessageImpl message) {
|
||||
NettyWritable encoded = new NettyWritable(Unpooled.buffer(1024));
|
||||
message.encode(encoded);
|
||||
|
||||
NettyReadable readable = new NettyReadable(encoded.getByteBuf());
|
||||
|
||||
return new AMQPMessage(AMQPMessage.DEFAULT_MESSAGE_FORMAT, readable, null, null);
|
||||
return new AMQPStandardMessage(AMQPMessage.DEFAULT_MESSAGE_FORMAT, readable, null, null);
|
||||
}
|
||||
|
||||
private void encode(AMQPMessage target) {
|
||||
|
@ -27,6 +27,7 @@ import java.util.Map;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPStandardMessage;
|
||||
import org.apache.activemq.artemis.protocol.amqp.converter.AMQPConverter;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyReadable;
|
||||
import org.apache.activemq.artemis.protocol.amqp.util.NettyWritable;
|
||||
@ -58,7 +59,7 @@ public class MessageTransformationTest {
|
||||
incomingMessage.setBody(new AmqpValue("String payload for AMQP message conversion performance testing."));
|
||||
|
||||
ICoreMessage core = encodeAndCreateAMQPMessage(incomingMessage).toCore();
|
||||
AMQPMessage outboudMessage = AMQPConverter.getInstance().fromCore(core);
|
||||
AMQPMessage outboudMessage = AMQPConverter.getInstance().fromCore(core, null);
|
||||
|
||||
assertNull(outboudMessage.getHeader());
|
||||
|
||||
@ -76,7 +77,7 @@ public class MessageTransformationTest {
|
||||
incomingMessage.setMessageId("ID:SomeQualifier:0:0:1");
|
||||
|
||||
ICoreMessage core = encodeAndCreateAMQPMessage(incomingMessage).toCore();
|
||||
AMQPMessage outboudMessage = AMQPConverter.getInstance().fromCore(core);
|
||||
AMQPMessage outboudMessage = AMQPConverter.getInstance().fromCore(core, null);
|
||||
|
||||
assertNull(outboudMessage.getHeader());
|
||||
assertNotNull(outboudMessage.getProperties());
|
||||
@ -90,7 +91,7 @@ public class MessageTransformationTest {
|
||||
incomingMessage.setDurable(true);
|
||||
|
||||
ICoreMessage core = encodeAndCreateAMQPMessage(incomingMessage).toCore();
|
||||
AMQPMessage outboudMessage = AMQPConverter.getInstance().fromCore(core);
|
||||
AMQPMessage outboudMessage = AMQPConverter.getInstance().fromCore(core, null);
|
||||
|
||||
assertNotNull(outboudMessage.getHeader());
|
||||
|
||||
@ -145,7 +146,7 @@ public class MessageTransformationTest {
|
||||
message.setBody(new AmqpValue("String payload for AMQP message conversion performance testing."));
|
||||
|
||||
ICoreMessage core = encodeAndCreateAMQPMessage(message).toCore();
|
||||
AMQPMessage outboudMessage = AMQPConverter.getInstance().fromCore(core);
|
||||
AMQPMessage outboudMessage = AMQPConverter.getInstance().fromCore(core, null);
|
||||
|
||||
assertEquals(10, outboudMessage.getApplicationProperties().getValue().size());
|
||||
assertEquals(4, outboudMessage.getMessageAnnotations().getValue().size());
|
||||
@ -157,6 +158,6 @@ public class MessageTransformationTest {
|
||||
|
||||
NettyReadable readable = new NettyReadable(encoded.getByteBuf());
|
||||
|
||||
return new AMQPMessage(AMQPMessage.DEFAULT_MESSAGE_FORMAT, readable, null, null);
|
||||
return new AMQPStandardMessage(AMQPMessage.DEFAULT_MESSAGE_FORMAT, readable, null, null);
|
||||
}
|
||||
}
|
||||
|
@ -19,7 +19,6 @@ package org.apache.activemq.artemis.protocol.amqp.proton;
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyInt;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.ArgumentMatchers.nullable;
|
||||
import static org.mockito.Mockito.doAnswer;
|
||||
@ -37,6 +36,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.server.RoutingContext;
|
||||
import org.apache.activemq.artemis.core.transaction.Transaction;
|
||||
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.broker.ProtonProtocolManager;
|
||||
import org.apache.activemq.artemis.protocol.amqp.exceptions.ActiveMQAMQPException;
|
||||
@ -48,7 +48,6 @@ import org.apache.qpid.proton.amqp.messaging.Outcome;
|
||||
import org.apache.qpid.proton.amqp.messaging.Rejected;
|
||||
import org.apache.qpid.proton.amqp.messaging.Source;
|
||||
import org.apache.qpid.proton.amqp.transport.DeliveryState;
|
||||
import org.apache.qpid.proton.codec.ReadableBuffer;
|
||||
import org.apache.qpid.proton.engine.Delivery;
|
||||
import org.apache.qpid.proton.engine.Receiver;
|
||||
import org.junit.Test;
|
||||
@ -68,40 +67,24 @@ public class ProtonServerReceiverContextTest {
|
||||
|
||||
@Test
|
||||
public void addressFull_SourceSupportsModified() throws Exception {
|
||||
doOnMessageWithDeliveryException(asList(Rejected.DESCRIPTOR_SYMBOL,
|
||||
Accepted.DESCRIPTOR_SYMBOL,
|
||||
Modified.DESCRIPTOR_SYMBOL),
|
||||
null, new ActiveMQAddressFullException(),
|
||||
Modified.class);
|
||||
doOnMessageWithDeliveryException(asList(Rejected.DESCRIPTOR_SYMBOL, Accepted.DESCRIPTOR_SYMBOL, Modified.DESCRIPTOR_SYMBOL), null, new ActiveMQAddressFullException(), Modified.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void addressFull_SourceDoesNotSupportModified() throws Exception {
|
||||
doOnMessageWithDeliveryException(asList(Rejected.DESCRIPTOR_SYMBOL,
|
||||
Accepted.DESCRIPTOR_SYMBOL),
|
||||
null, new ActiveMQAddressFullException(),
|
||||
Rejected.class);
|
||||
doOnMessageWithDeliveryException(asList(Rejected.DESCRIPTOR_SYMBOL, Accepted.DESCRIPTOR_SYMBOL), null, new ActiveMQAddressFullException(), Rejected.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void otherFailure_SourceSupportsRejects() throws Exception {
|
||||
doOnMessageWithDeliveryException(asList(Rejected.DESCRIPTOR_SYMBOL,
|
||||
Accepted.DESCRIPTOR_SYMBOL,
|
||||
Modified.DESCRIPTOR_SYMBOL),
|
||||
null, new ActiveMQException(),
|
||||
Rejected.class);
|
||||
doOnMessageWithDeliveryException(asList(Rejected.DESCRIPTOR_SYMBOL, Accepted.DESCRIPTOR_SYMBOL, Modified.DESCRIPTOR_SYMBOL), null, new ActiveMQException(), Rejected.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void otherFailure_SourceDoesNotSupportReject() throws Exception {
|
||||
doOnMessageWithDeliveryException(singletonList(Accepted.DESCRIPTOR_SYMBOL),
|
||||
Accepted.getInstance(), new ActiveMQException(),
|
||||
Accepted.class);
|
||||
doOnMessageWithDeliveryException(singletonList(Accepted.DESCRIPTOR_SYMBOL), Accepted.getInstance(), new ActiveMQException(), Accepted.class);
|
||||
// violates AMQP specification - see explanation ProtonServerReceiverContext.determineDeliveryState
|
||||
doOnMessageWithDeliveryException(singletonList(Accepted.DESCRIPTOR_SYMBOL),
|
||||
null,
|
||||
new ActiveMQException(),
|
||||
Rejected.class);
|
||||
doOnMessageWithDeliveryException(singletonList(Accepted.DESCRIPTOR_SYMBOL), null, new ActiveMQException(), Rejected.class);
|
||||
}
|
||||
|
||||
private void doOnMessageWithAbortedDeliveryTestImpl(boolean drain) throws ActiveMQAMQPException {
|
||||
@ -140,7 +123,8 @@ public class ProtonServerReceiverContextTest {
|
||||
}
|
||||
|
||||
private void doOnMessageWithDeliveryException(List<Symbol> sourceSymbols,
|
||||
Outcome defaultOutcome, Exception deliveryException,
|
||||
Outcome defaultOutcome,
|
||||
Exception deliveryException,
|
||||
Class<? extends DeliveryState> expectedDeliveryState) throws Exception {
|
||||
AMQPConnectionContext mockConnContext = mock(AMQPConnectionContext.class);
|
||||
doAnswer((Answer<Void>) invocation -> {
|
||||
@ -152,7 +136,6 @@ public class ProtonServerReceiverContextTest {
|
||||
when(mockProtocolManager.isUseModifiedForTransientDeliveryErrors()).thenReturn(true);
|
||||
when(mockConnContext.getProtocolManager()).thenReturn(mockProtocolManager);
|
||||
|
||||
|
||||
AMQPSessionCallback mockSession = mock(AMQPSessionCallback.class);
|
||||
|
||||
Receiver mockReceiver = mock(Receiver.class);
|
||||
@ -167,15 +150,7 @@ public class ProtonServerReceiverContextTest {
|
||||
source.setDefaultOutcome(defaultOutcome);
|
||||
when(mockReceiver.getSource()).thenReturn(source);
|
||||
|
||||
doThrow(deliveryException).when(mockSession)
|
||||
.serverSend(eq(rc),
|
||||
nullable(Transaction.class),
|
||||
eq(mockReceiver),
|
||||
eq(mockDelivery),
|
||||
nullable(SimpleString.class),
|
||||
anyInt(),
|
||||
nullable(ReadableBuffer.class),
|
||||
any(RoutingContext.class));
|
||||
doThrow(deliveryException).when(mockSession).serverSend(eq(rc), nullable(Transaction.class), eq(mockReceiver), eq(mockDelivery), nullable(SimpleString.class), any(RoutingContext.class), nullable(AMQPMessage.class));
|
||||
|
||||
rc.onMessage(mockDelivery);
|
||||
|
||||
|
@ -20,7 +20,7 @@ package org.apache.activemq.artemis.core.protocol.mqtt;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.activemq.artemis.core.config.WildcardConfiguration;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
import org.apache.activemq.artemis.core.server.impl.ServerSessionImpl;
|
||||
import org.apache.activemq.artemis.spi.core.protocol.SessionCallback;
|
||||
|
@ -42,7 +42,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQPropertyConversionException;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.amq.AMQConsumer;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.util.OpenWireUtil;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
|
@ -24,9 +24,8 @@ import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
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.RefCountMessageListener;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
@ -69,21 +68,16 @@ public class OpenwireMessage implements Message {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RefCountMessageListener getContext() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message setContext(RefCountMessageListener context) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message setBuffer(ByteBuf buffer) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getDurableCount() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuf getBuffer() {
|
||||
return null;
|
||||
@ -140,7 +134,7 @@ public class OpenwireMessage implements Message {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Persister<Message, CoreMessageObjectPools> getPersister() {
|
||||
public Persister<Message> getPersister() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@ -465,22 +459,22 @@ public class OpenwireMessage implements Message {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementRefCount() throws Exception {
|
||||
public int refUp() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementRefCount() throws Exception {
|
||||
public int refDown() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementDurableRefCount() {
|
||||
public int durableUp() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementDurableRefCount() {
|
||||
public int durableDown() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@ -503,4 +497,19 @@ public class OpenwireMessage implements Message {
|
||||
public long getPersistentSize() throws ActiveMQException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUsage() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int usageUp() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int usageDown() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
@ -32,7 +32,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQQueueExistsException;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.io.IOCallback;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireConnection;
|
||||
import org.apache.activemq.artemis.core.protocol.openwire.OpenWireMessageConverter;
|
||||
|
@ -363,11 +363,9 @@ public class StompSession implements SessionCallback {
|
||||
|
||||
largeMessage.releaseResources(true);
|
||||
|
||||
largeMessage.putLongProperty(Message.HDR_LARGE_BODY_SIZE, bytes.length);
|
||||
largeMessage.toMessage().putLongProperty(Message.HDR_LARGE_BODY_SIZE, bytes.length);
|
||||
|
||||
session.send(largeMessage, direct);
|
||||
|
||||
largeMessage = null;
|
||||
session.send(largeMessage.toMessage(), direct);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,6 @@ package org.apache.activemq.artemis.core.paging.cursor.impl;
|
||||
import org.apache.activemq.artemis.core.paging.PagedMessage;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.LivePageCache;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.apache.activemq.artemis.utils.collections.ConcurrentAppendOnlyChunkedList;
|
||||
import org.jboss.logging.Logger;
|
||||
|
||||
@ -73,9 +72,7 @@ public final class LivePageCacheImpl implements LivePageCache {
|
||||
|
||||
@Override
|
||||
public void addLiveMessage(PagedMessage message) {
|
||||
if (message.getMessage().isLargeMessage()) {
|
||||
((LargeServerMessage) message.getMessage()).incrementDelayDeletionCount();
|
||||
}
|
||||
message.getMessage().usageUp();
|
||||
messages.add(message);
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.buffer.Unpooled;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFile;
|
||||
@ -35,7 +34,6 @@ import org.apache.activemq.artemis.core.paging.cursor.PageSubscriptionCounter;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
import org.apache.activemq.artemis.utils.Env;
|
||||
import org.apache.activemq.artemis.utils.collections.ConcurrentHashSet;
|
||||
@ -494,14 +492,10 @@ public final class Page implements Comparable<Page> {
|
||||
List<Long> largeMessageIds = new ArrayList<>();
|
||||
if (messages != null) {
|
||||
for (PagedMessage msg : messages) {
|
||||
if (msg.getMessage() instanceof ICoreMessage && (msg.getMessage()).isLargeMessage()) {
|
||||
LargeServerMessage lmsg = (LargeServerMessage) msg.getMessage();
|
||||
|
||||
// Remember, cannot call delete directly here
|
||||
// Because the large-message may be linked to another message
|
||||
// or it may still being delivered even though it has been acked already
|
||||
lmsg.decrementDelayDeletionCount();
|
||||
largeMessageIds.add(lmsg.getMessageID());
|
||||
// this will trigger large message delete
|
||||
msg.getMessage().usageDown();
|
||||
if ((msg.getMessage()).isLargeMessage()) {
|
||||
largeMessageIds.add(msg.getMessage().getMessageID());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -74,10 +74,10 @@ public class PagedMessageImpl implements PagedMessage {
|
||||
LargeServerMessage lgMessage = storage.createLargeMessage();
|
||||
|
||||
ActiveMQBuffer buffer = ActiveMQBuffers.wrappedBuffer(largeMessageLazyData);
|
||||
lgMessage = LargeMessagePersister.getInstance().decode(buffer, lgMessage);
|
||||
lgMessage.incrementDelayDeletionCount();
|
||||
lgMessage = LargeMessagePersister.getInstance().decode(buffer, lgMessage, null);
|
||||
lgMessage.toMessage().usageUp();
|
||||
lgMessage.setPaged();
|
||||
this.message = lgMessage;
|
||||
this.message = lgMessage.toMessage();
|
||||
largeMessageLazyData = null;
|
||||
}
|
||||
}
|
||||
@ -107,12 +107,12 @@ public class PagedMessageImpl implements PagedMessage {
|
||||
largeMessageLazyData = new byte[largeMessageHeaderSize];
|
||||
buffer.readBytes(largeMessageLazyData);
|
||||
} else {
|
||||
this.message = storageManager.createLargeMessage();
|
||||
LargeMessagePersister.getInstance().decode(buffer, (LargeServerMessage) message);
|
||||
((LargeServerMessage) message).incrementDelayDeletionCount();
|
||||
this.message = storageManager.createLargeMessage().toMessage();
|
||||
LargeMessagePersister.getInstance().decode(buffer, (LargeServerMessage) message, null);
|
||||
((LargeServerMessage) message).toMessage().usageUp();
|
||||
}
|
||||
} else {
|
||||
this.message = MessagePersister.getInstance().decode(buffer, null);
|
||||
this.message = MessagePersister.getInstance().decode(buffer, null, null);
|
||||
}
|
||||
|
||||
int queueIDsSize = buffer.readInt();
|
||||
|
@ -446,37 +446,7 @@ public class PagingStoreImpl implements PagingStore {
|
||||
currentPageId = pageId;
|
||||
|
||||
if (pageId != 0) {
|
||||
Page page = createPage(pageId);
|
||||
page.open();
|
||||
|
||||
List<PagedMessage> messages = page.read(storageManager);
|
||||
|
||||
LivePageCache pageCache = new LivePageCacheImpl(pageId);
|
||||
|
||||
for (PagedMessage msg : messages) {
|
||||
pageCache.addLiveMessage(msg);
|
||||
if (msg.getMessage().isLargeMessage()) {
|
||||
// We have to do this since addLIveMessage will increment an extra one
|
||||
((LargeServerMessage) msg.getMessage()).decrementDelayDeletionCount();
|
||||
}
|
||||
}
|
||||
|
||||
page.setLiveCache(pageCache);
|
||||
|
||||
currentPageSize = page.getSize();
|
||||
|
||||
currentPage = page;
|
||||
|
||||
cursorProvider.addPageCache(pageCache);
|
||||
|
||||
/**
|
||||
* The page file might be incomplete in the cases: 1) last message incomplete 2) disk damaged.
|
||||
* In case 1 we can keep writing the file. But in case 2 we'd better not bcs old data might be overwritten.
|
||||
* Here we open a new page so the incomplete page would be reserved for recovery if needed.
|
||||
*/
|
||||
if (page.getSize() != page.getFile().size()) {
|
||||
openNewPage();
|
||||
}
|
||||
reloadLivePage(pageId);
|
||||
}
|
||||
|
||||
// We will not mark it for paging if there's only a single empty file
|
||||
@ -492,6 +462,39 @@ public class PagingStoreImpl implements PagingStore {
|
||||
}
|
||||
}
|
||||
|
||||
protected void reloadLivePage(int pageId) throws Exception {
|
||||
Page page = createPage(pageId);
|
||||
page.open();
|
||||
|
||||
List<PagedMessage> messages = page.read(storageManager);
|
||||
|
||||
LivePageCache pageCache = new LivePageCacheImpl(pageId);
|
||||
|
||||
for (PagedMessage msg : messages) {
|
||||
pageCache.addLiveMessage(msg);
|
||||
// As we add back to the live page,
|
||||
// we have to discount one when we read the page
|
||||
msg.getMessage().usageDown();
|
||||
}
|
||||
|
||||
page.setLiveCache(pageCache);
|
||||
|
||||
currentPageSize = page.getSize();
|
||||
|
||||
currentPage = page;
|
||||
|
||||
cursorProvider.addPageCache(pageCache);
|
||||
|
||||
/**
|
||||
* The page file might be incomplete in the cases: 1) last message incomplete 2) disk damaged.
|
||||
* In case 1 we can keep writing the file. But in case 2 we'd better not bcs old data might be overwritten.
|
||||
* Here we open a new page so the incomplete page would be reserved for recovery if needed.
|
||||
*/
|
||||
if (page.getSize() != page.getFile().size()) {
|
||||
openNewPage();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopPaging() {
|
||||
lock.writeLock().lock();
|
||||
@ -953,14 +956,16 @@ public class PagingStoreImpl implements PagingStore {
|
||||
|
||||
@Override
|
||||
public void durableDown(Message message, int durableCount) {
|
||||
refDown(message, durableCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void durableUp(Message message, int durableCount) {
|
||||
refUp(message, durableCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void nonDurableUp(Message message, int count) {
|
||||
public void refUp(Message message, int count) {
|
||||
if (count == 1) {
|
||||
this.addSize(message.getMemoryEstimate() + MessageReferenceImpl.getMemoryEstimate());
|
||||
} else {
|
||||
@ -969,9 +974,9 @@ public class PagingStoreImpl implements PagingStore {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void nonDurableDown(Message message, int count) {
|
||||
public void refDown(Message message, int count) {
|
||||
if (count < 0) {
|
||||
// this could happen on paged messages since they are not routed and incrementRefCount is never called
|
||||
// this could happen on paged messages since they are not routed and refUp is never called
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -24,6 +24,8 @@ import java.util.Set;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
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.Pair;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
@ -265,6 +267,17 @@ public interface StorageManager extends IDGenerator, ActiveMQComponent {
|
||||
*/
|
||||
SequentialFile createFileForLargeMessage(long messageID, LargeMessageExtension extension);
|
||||
|
||||
void deleteLargeMessageBody(LargeServerMessage largeServerMessage) throws ActiveMQException;
|
||||
|
||||
default SequentialFile createFileForLargeMessage(long messageID, boolean durable) {
|
||||
if (durable) {
|
||||
return createFileForLargeMessage(messageID, LargeMessageExtension.DURABLE);
|
||||
} else {
|
||||
return createFileForLargeMessage(messageID, LargeMessageExtension.TEMPORARY);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void prepare(long txID, Xid xid) throws Exception;
|
||||
|
||||
void commit(long txID) throws Exception;
|
||||
@ -416,6 +429,10 @@ public interface StorageManager extends IDGenerator, ActiveMQComponent {
|
||||
*/
|
||||
void addBytesToLargeMessage(SequentialFile appendFile, long messageID, byte[] bytes) throws Exception;
|
||||
|
||||
void addBytesToLargeMessage(SequentialFile file,
|
||||
long messageId,
|
||||
ActiveMQBuffer bytes) throws Exception;
|
||||
|
||||
/**
|
||||
* Stores the id from IDManager.
|
||||
*
|
||||
|
@ -58,7 +58,7 @@ import org.apache.activemq.artemis.core.journal.Journal;
|
||||
import org.apache.activemq.artemis.core.journal.JournalLoadInformation;
|
||||
import org.apache.activemq.artemis.core.journal.PreparedTransactionInfo;
|
||||
import org.apache.activemq.artemis.core.journal.RecordInfo;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.paging.PageTransactionInfo;
|
||||
import org.apache.activemq.artemis.core.paging.PagingManager;
|
||||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
@ -134,6 +134,19 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
protected static final int CRITICAL_STOP = 1;
|
||||
protected static final int CRITICAL_STOP_2 = 2;
|
||||
|
||||
|
||||
public static ThreadLocal<StorageManager> storageManagerThreadLocal = new ThreadLocal<>();
|
||||
|
||||
/** Persisters may need to access this on reloading of the journal,
|
||||
* for large message processing */
|
||||
public static void setupThreadLocal(StorageManager manager) {
|
||||
storageManagerThreadLocal.set(manager);
|
||||
}
|
||||
|
||||
public static StorageManager getThreadLocal() {
|
||||
return storageManagerThreadLocal.get();
|
||||
}
|
||||
|
||||
private static final Logger logger = Logger.getLogger(AbstractJournalStorageManager.class);
|
||||
|
||||
public enum JournalContent {
|
||||
@ -359,7 +372,7 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
// Note that we don't sync, the add reference that comes immediately after will sync if
|
||||
// appropriate
|
||||
|
||||
if (message.isLargeMessage()) {
|
||||
if (message.isLargeMessage() && message instanceof LargeServerMessageImpl) {
|
||||
messageJournal.appendAddRecord(message.getMessageID(), JournalRecordIds.ADD_LARGE_MESSAGE, LargeMessagePersister.getInstance(), message, false, getContext(false));
|
||||
} else {
|
||||
messageJournal.appendAddRecord(message.getMessageID(), JournalRecordIds.ADD_MESSAGE_PROTOCOL, message.getPersister(), message, false, getContext(false));
|
||||
@ -480,7 +493,8 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
|
||||
readLock();
|
||||
try {
|
||||
if (message.isLargeMessage()) {
|
||||
if (message.isLargeMessage() && message instanceof LargeServerMessageImpl) {
|
||||
// this is a core large message
|
||||
messageJournal.appendAddRecordTransactional(txID, message.getMessageID(), JournalRecordIds.ADD_LARGE_MESSAGE, LargeMessagePersister.getInstance(), message);
|
||||
} else {
|
||||
messageJournal.appendAddRecordTransactional(txID, message.getMessageID(), JournalRecordIds.ADD_MESSAGE_PROTOCOL, message.getPersister(), message);
|
||||
@ -843,6 +857,7 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
|
||||
Map<Long, Message> messages = new HashMap<>();
|
||||
readLock();
|
||||
setupThreadLocal(this);
|
||||
try {
|
||||
|
||||
JournalLoadInformation info = messageJournal.load(records, preparedTransactions, new LargeMessageTXFailureCallback(this));
|
||||
@ -908,7 +923,7 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
case JournalRecordIds.ADD_LARGE_MESSAGE: {
|
||||
LargeServerMessage largeMessage = parseLargeMessage(buff);
|
||||
|
||||
messages.put(record.id, largeMessage);
|
||||
messages.put(record.id, largeMessage.toMessage());
|
||||
|
||||
largeMessages.add(largeMessage);
|
||||
|
||||
@ -920,7 +935,15 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
|
||||
case JournalRecordIds.ADD_MESSAGE_PROTOCOL: {
|
||||
|
||||
Message message = MessagePersister.getInstance().decode(buff, pools);
|
||||
Message message = MessagePersister.getInstance().decode(buff, null, pools);
|
||||
|
||||
/* if (message instanceof LargeServerMessage) {
|
||||
try {
|
||||
((LargeServerMessage) message).finishParse();
|
||||
} catch (Exception e) {
|
||||
logger.warn(e.getMessage(), e);
|
||||
}
|
||||
} */
|
||||
|
||||
messages.put(record.id, message);
|
||||
|
||||
@ -1194,9 +1217,9 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
}
|
||||
|
||||
for (LargeServerMessage msg : largeMessages) {
|
||||
if (msg.getRefCount() == 0) {
|
||||
if (msg.toMessage().getRefCount() == 0) {
|
||||
ActiveMQServerLogger.LOGGER.largeMessageWithNoRef(msg.getMessageID());
|
||||
msg.decrementDelayDeletionCount();
|
||||
msg.toMessage().usageDown();
|
||||
}
|
||||
}
|
||||
|
||||
@ -1217,6 +1240,8 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
return info;
|
||||
} finally {
|
||||
readUnLock();
|
||||
// need to clear it, otherwise we may have a permanent leak
|
||||
setupThreadLocal(null);
|
||||
}
|
||||
}
|
||||
|
||||
@ -1712,7 +1737,7 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
if (largeServerMessage.getPendingRecordID() >= 0) {
|
||||
try {
|
||||
confirmPendingLargeMessage(largeServerMessage.getPendingRecordID());
|
||||
largeServerMessage.setPendingRecordID(LargeServerMessage.NO_PENDING_ID);
|
||||
largeServerMessage.clearPendingRecordID();
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.warn(e.getMessage(), e);
|
||||
}
|
||||
@ -1758,7 +1783,7 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
|
||||
switch (recordType) {
|
||||
case JournalRecordIds.ADD_LARGE_MESSAGE: {
|
||||
messages.put(record.id, parseLargeMessage(buff));
|
||||
messages.put(record.id, parseLargeMessage(buff).toMessage());
|
||||
|
||||
break;
|
||||
}
|
||||
@ -1770,7 +1795,7 @@ public abstract class AbstractJournalStorageManager extends CriticalComponentImp
|
||||
if (pools == null) {
|
||||
pools = new CoreMessageObjectPools();
|
||||
}
|
||||
Message message = MessagePersister.getInstance().decode(buff, pools);
|
||||
Message message = MessagePersister.getInstance().decode(buff, null, pools);
|
||||
|
||||
messages.put(record.id, message);
|
||||
|
||||
|
@ -561,16 +561,15 @@ public final class DescribeJournal {
|
||||
|
||||
LargeServerMessage largeMessage = new LargeServerMessageImpl(storageManager);
|
||||
|
||||
LargeMessagePersister.getInstance().decode(buffer, largeMessage);
|
||||
LargeMessagePersister.getInstance().decode(buffer, largeMessage, null);
|
||||
|
||||
return new MessageDescribe(largeMessage);
|
||||
return new MessageDescribe(largeMessage.toMessage());
|
||||
}
|
||||
case ADD_MESSAGE: {
|
||||
return "ADD-MESSAGE is not supported any longer, use export/import";
|
||||
}
|
||||
case ADD_MESSAGE_PROTOCOL: {
|
||||
Message message = MessagePersister.getInstance().decode(buffer, null);
|
||||
|
||||
Message message = MessagePersister.getInstance().decode(buffer, null, null);
|
||||
return new MessageDescribe(message);
|
||||
}
|
||||
case ADD_REF: {
|
||||
|
@ -348,14 +348,6 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
|
||||
}
|
||||
}
|
||||
|
||||
protected SequentialFile createFileForLargeMessage(final long messageID, final boolean durable) {
|
||||
if (durable) {
|
||||
return createFileForLargeMessage(messageID, LargeMessageExtension.DURABLE);
|
||||
} else {
|
||||
return createFileForLargeMessage(messageID, LargeMessageExtension.TEMPORARY);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
* @param buff
|
||||
@ -365,13 +357,13 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
|
||||
protected LargeServerMessage parseLargeMessage(final ActiveMQBuffer buff) throws Exception {
|
||||
LargeServerMessage largeMessage = createLargeMessage();
|
||||
|
||||
LargeMessagePersister.getInstance().decode(buff, largeMessage);
|
||||
LargeMessagePersister.getInstance().decode(buff, largeMessage, null);
|
||||
|
||||
if (largeMessage.containsProperty(Message.HDR_ORIG_MESSAGE_ID)) {
|
||||
if (largeMessage.toMessage().containsProperty(Message.HDR_ORIG_MESSAGE_ID)) {
|
||||
// for compatibility: couple with old behaviour, copying the old file to avoid message loss
|
||||
long originalMessageID = largeMessage.getLongProperty(Message.HDR_ORIG_MESSAGE_ID);
|
||||
long originalMessageID = largeMessage.toMessage().getLongProperty(Message.HDR_ORIG_MESSAGE_ID);
|
||||
|
||||
SequentialFile currentFile = createFileForLargeMessage(largeMessage.getMessageID(), true);
|
||||
SequentialFile currentFile = createFileForLargeMessage(largeMessage.toMessage().getMessageID(), true);
|
||||
|
||||
if (!currentFile.exists()) {
|
||||
SequentialFile linkedFile = createFileForLargeMessage(originalMessageID, true);
|
||||
@ -442,18 +434,10 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
|
||||
journalFF.releaseBuffer(buffer);
|
||||
}
|
||||
|
||||
public long storePendingLargeMessage(final long messageID, long recordID) throws Exception {
|
||||
public long storePendingLargeMessage(final long messageID) throws Exception {
|
||||
readLock();
|
||||
try {
|
||||
if (recordID == LargeServerMessage.NO_PENDING_ID) {
|
||||
recordID = generateID();
|
||||
} else {
|
||||
//this means the large message doesn't
|
||||
//have a pendingRecordID, but one has been
|
||||
//generated (coming from live server) for use.
|
||||
recordID = -recordID;
|
||||
}
|
||||
|
||||
long recordID = generateID();
|
||||
messageJournal.appendAddRecord(recordID, JournalRecordIds.ADD_LARGE_MESSAGE_PENDING, new PendingLargeMessageEncoding(messageID), true, getContext(true));
|
||||
|
||||
return recordID;
|
||||
@ -462,31 +446,31 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
|
||||
}
|
||||
}
|
||||
|
||||
// This should be accessed from this package only
|
||||
void deleteLargeMessageFile(final LargeServerMessage largeServerMessage) throws ActiveMQException {
|
||||
@Override
|
||||
public void deleteLargeMessageBody(final LargeServerMessage largeServerMessage) throws ActiveMQException {
|
||||
synchronized (largeServerMessage) {
|
||||
if (largeServerMessage.getPendingRecordID() < 0) {
|
||||
if (!largeServerMessage.hasPendingRecord()) {
|
||||
try {
|
||||
// The delete file happens asynchronously
|
||||
// And the client won't be waiting for the actual file to be deleted.
|
||||
// We set a temporary record (short lived) on the journal
|
||||
// to avoid a situation where the server is restarted and pending large message stays on forever
|
||||
largeServerMessage.setPendingRecordID(storePendingLargeMessage(largeServerMessage.getMessageID(), largeServerMessage.getPendingRecordID()));
|
||||
largeServerMessage.setPendingRecordID(storePendingLargeMessage(largeServerMessage.toMessage().getMessageID()));
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQInternalErrorException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
final SequentialFile file = largeServerMessage.getFile();
|
||||
final SequentialFile file = largeServerMessage.getAppendFile();
|
||||
if (file == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (largeServerMessage.isDurable() && isReplicated()) {
|
||||
if (largeServerMessage.toMessage().isDurable() && isReplicated()) {
|
||||
readLock();
|
||||
try {
|
||||
if (isReplicated() && replicator.isSynchronizing()) {
|
||||
largeMessagesToDelete.put(largeServerMessage.getMessageID(), largeServerMessage);
|
||||
largeMessagesToDelete.put(largeServerMessage.toMessage().getMessageID(), largeServerMessage);
|
||||
return;
|
||||
}
|
||||
} finally {
|
||||
@ -500,7 +484,7 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
|
||||
readLock();
|
||||
try {
|
||||
if (replicator != null) {
|
||||
replicator.largeMessageDelete(largeServerMessage.getMessageID(), JournalStorageManager.this);
|
||||
replicator.largeMessageDelete(largeServerMessage.toMessage().getMessageID(), JournalStorageManager.this);
|
||||
}
|
||||
file.delete();
|
||||
|
||||
@ -510,7 +494,7 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
|
||||
readUnLock();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.journalErrorDeletingMessage(e, largeServerMessage.getMessageID());
|
||||
ActiveMQServerLogger.LOGGER.journalErrorDeletingMessage(e, largeServerMessage.toMessage().getMessageID());
|
||||
}
|
||||
}
|
||||
|
||||
@ -575,7 +559,7 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
|
||||
|
||||
if (largeMessage.isDurable()) {
|
||||
// We store a marker on the journal that the large file is pending
|
||||
long pendingRecordID = storePendingLargeMessage(id, LargeServerMessage.NO_PENDING_ID);
|
||||
long pendingRecordID = storePendingLargeMessage(id);
|
||||
|
||||
largeMessage.setPendingRecordID(pendingRecordID);
|
||||
}
|
||||
@ -831,6 +815,7 @@ public class JournalStorageManager extends AbstractJournalStorageManager {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void addBytesToLargeMessage(final SequentialFile file,
|
||||
final long messageId,
|
||||
final ActiveMQBuffer bytes) throws Exception {
|
||||
|
@ -0,0 +1,450 @@
|
||||
/*
|
||||
* 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.persistence.impl.journal;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import io.netty.buffer.Unpooled;
|
||||
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.ActiveMQIOErrorException;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQInternalErrorException;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFile;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.jboss.logging.Logger;
|
||||
|
||||
public class LargeBody {
|
||||
|
||||
private static final Logger logger = Logger.getLogger(LargeBody.class);
|
||||
|
||||
private long bodySize = -1;
|
||||
|
||||
long NO_PENDING_ID = -1;
|
||||
|
||||
private long pendingRecordID = NO_PENDING_ID;
|
||||
|
||||
final StorageManager storageManager;
|
||||
|
||||
private long messageID = -1;
|
||||
|
||||
private LargeServerMessage message;
|
||||
|
||||
private boolean paged;
|
||||
|
||||
// This is to be used only for appending
|
||||
private SequentialFile file;
|
||||
|
||||
public LargeBody(LargeServerMessage message, StorageManager storageManager) {
|
||||
this.storageManager = storageManager;
|
||||
this.message = message;
|
||||
}
|
||||
|
||||
public LargeBody(LargeServerMessage message, StorageManager storageManager, SequentialFile file) {
|
||||
this(message, storageManager);
|
||||
this.file = file;
|
||||
}
|
||||
|
||||
public StorageManager getStorageManager() {
|
||||
return storageManager;
|
||||
}
|
||||
|
||||
public ByteBuffer map() throws Exception {
|
||||
ensureFileExists(true);
|
||||
if (!file.isOpen()) {
|
||||
file.open();
|
||||
}
|
||||
return file.map(0, file.size());
|
||||
}
|
||||
|
||||
public LargeBody(long messageID, JournalStorageManager storageManager) {
|
||||
this(null, storageManager);
|
||||
this.messageID = messageID;
|
||||
}
|
||||
|
||||
public void setMessage(LargeServerMessage message) {
|
||||
this.message = message;
|
||||
|
||||
}
|
||||
|
||||
public void setPaged() {
|
||||
this.paged = true;
|
||||
}
|
||||
|
||||
public boolean isPaged() {
|
||||
return paged;
|
||||
}
|
||||
|
||||
public void clearFile() {
|
||||
if (file != null && file.isOpen()) {
|
||||
try {
|
||||
file.close();
|
||||
} catch (Exception e) {
|
||||
// this shouldn't happen anyways, this close call is here just in case it ever happened
|
||||
logger.warn(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
file = null;
|
||||
}
|
||||
|
||||
public synchronized void deleteFile() {
|
||||
try {
|
||||
validateFile();
|
||||
releaseResources(false);
|
||||
storageManager.deleteLargeMessageBody(message);
|
||||
} catch (Exception e) {
|
||||
storageManager.criticalError(e);
|
||||
}
|
||||
}
|
||||
|
||||
public long getMessageID() {
|
||||
if (message == null) {
|
||||
return messageID;
|
||||
} else {
|
||||
return message.getMessageID();
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void addBytes(final byte[] bytes) throws Exception {
|
||||
validateFile();
|
||||
|
||||
if (!file.isOpen()) {
|
||||
file.open();
|
||||
}
|
||||
|
||||
storageManager.addBytesToLargeMessage(file, getMessageID(), bytes);
|
||||
|
||||
bodySize += bytes.length;
|
||||
}
|
||||
|
||||
public synchronized void addBytes(final ActiveMQBuffer bytes) throws Exception {
|
||||
validateFile();
|
||||
|
||||
if (!file.isOpen()) {
|
||||
file.open();
|
||||
}
|
||||
|
||||
final int readableBytes = bytes.readableBytes();
|
||||
|
||||
storageManager.addBytesToLargeMessage(file, getMessageID(), bytes);
|
||||
|
||||
bodySize += readableBytes;
|
||||
}
|
||||
|
||||
public synchronized void validateFile() throws ActiveMQException {
|
||||
this.ensureFileExists(true);
|
||||
}
|
||||
|
||||
public synchronized void ensureFileExists(boolean toOpen) throws ActiveMQException {
|
||||
try {
|
||||
if (file == null) {
|
||||
if (getMessageID() <= 0) {
|
||||
throw new RuntimeException("MessageID not set on LargeMessage");
|
||||
}
|
||||
|
||||
file = createFile();
|
||||
|
||||
if (toOpen) {
|
||||
openFile();
|
||||
}
|
||||
|
||||
bodySize = file.size();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQInternalErrorException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This will return the bodySize without trying to open the file, just returning what's currently stored
|
||||
*/
|
||||
public long getStoredBodySize() {
|
||||
return bodySize;
|
||||
}
|
||||
|
||||
public void setBodySize(long size) {
|
||||
this.bodySize = size;
|
||||
}
|
||||
|
||||
public long getBodySize() throws ActiveMQException {
|
||||
|
||||
try {
|
||||
if (bodySize <= 0) {
|
||||
if (file != null) {
|
||||
bodySize = file.size();
|
||||
} else {
|
||||
SequentialFile tmpFile = createFile();
|
||||
bodySize = tmpFile.size();
|
||||
tmpFile.close(false);
|
||||
}
|
||||
}
|
||||
return bodySize;
|
||||
} catch (Exception e) {
|
||||
ActiveMQIOErrorException errorException = new ActiveMQIOErrorException();
|
||||
errorException.initCause(e);
|
||||
throw errorException;
|
||||
}
|
||||
}
|
||||
|
||||
public LargeBodyReader getLargeBodyReader() {
|
||||
return new LargeBodyReaderImpl();
|
||||
}
|
||||
|
||||
/**
|
||||
* This will return its own File useful for reading the file on the large message while delivering, browsing.. etc
|
||||
*/
|
||||
public SequentialFile getReadingFile() throws ActiveMQException {
|
||||
ensureFileExists(false);
|
||||
return file.cloneFile();
|
||||
}
|
||||
|
||||
/** Meant for test-ability, be careful if you decide to use it.
|
||||
* and in case you use it for a real reason, please change the documentation here.
|
||||
* @param file
|
||||
*/
|
||||
public void replaceFile(SequentialFile file) {
|
||||
this.file = file;
|
||||
}
|
||||
|
||||
public SequentialFile getAppendFile() throws ActiveMQException {
|
||||
validateFile();
|
||||
return file;
|
||||
}
|
||||
|
||||
public void checkDelete() {
|
||||
if (message.toMessage().getRefCount() <= 0 && message.toMessage().getUsage() <= 0 && message.toMessage().getDurableCount() <= 0) {
|
||||
if (logger.isTraceEnabled()) {
|
||||
try {
|
||||
logger.trace("Deleting file " + getAppendFile() + " as the usage was complete");
|
||||
} catch (Exception e) {
|
||||
// this is only after a trace, no need to do any special logging handling here
|
||||
logger.warn(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
deleteFile();
|
||||
}
|
||||
}
|
||||
|
||||
public void referenceOriginalMessage(final LargeBody original) {
|
||||
if (original.isPaged()) {
|
||||
this.setPaged();
|
||||
}
|
||||
|
||||
if (this.paged) {
|
||||
message.toMessage().removeAnnotation(Message.HDR_ORIG_MESSAGE_ID);
|
||||
}
|
||||
}
|
||||
|
||||
public ActiveMQBuffer getReadOnlyBodyBuffer() {
|
||||
try {
|
||||
validateFile();
|
||||
file.open();
|
||||
int fileSize = (int) file.size();
|
||||
ByteBuffer buffer = ByteBuffer.allocate(fileSize);
|
||||
file.read(buffer);
|
||||
return new ChannelBufferWrapper(Unpooled.wrappedBuffer(buffer));
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
} finally {
|
||||
try {
|
||||
file.close(false);
|
||||
} catch (Exception ignored) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public int getBodyBufferSize() {
|
||||
final boolean closeFile = file == null || !file.isOpen();
|
||||
try {
|
||||
openFile();
|
||||
final long fileSize = file.size();
|
||||
int fileSizeAsInt = (int) fileSize;
|
||||
if (fileSizeAsInt < 0) {
|
||||
logger.warnf("suspicious large message file size of %d bytes for %s, will use %d instead.", fileSize, file.getFileName(), Integer.MAX_VALUE);
|
||||
fileSizeAsInt = Integer.MAX_VALUE;
|
||||
}
|
||||
return fileSizeAsInt;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
} finally {
|
||||
if (closeFile) {
|
||||
try {
|
||||
file.close(false);
|
||||
} catch (Exception ignored) {
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void releaseResources(boolean sync) {
|
||||
if (file != null && file.isOpen()) {
|
||||
try {
|
||||
if (sync) {
|
||||
file.sync();
|
||||
}
|
||||
file.close(false);
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.largeMessageErrorReleasingResources(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void copyInto(LargeServerMessage newMessage) throws Exception {
|
||||
//clone a SequentialFile to avoid concurrent access
|
||||
SequentialFile cloneFile = getReadingFile();
|
||||
|
||||
try {
|
||||
byte[] bufferBytes = new byte[100 * 1024];
|
||||
|
||||
ByteBuffer buffer = ByteBuffer.wrap(bufferBytes);
|
||||
|
||||
if (!cloneFile.isOpen()) {
|
||||
cloneFile.open();
|
||||
}
|
||||
|
||||
cloneFile.position(0);
|
||||
|
||||
for (; ; ) {
|
||||
// The buffer is reused...
|
||||
// We need to make sure we clear the limits and the buffer before reusing it
|
||||
buffer.clear();
|
||||
int bytesRead = cloneFile.read(buffer);
|
||||
|
||||
byte[] bufferToWrite;
|
||||
if (bytesRead <= 0) {
|
||||
break;
|
||||
} else if (bytesRead == bufferBytes.length && this.storageManager instanceof JournalStorageManager && !((JournalStorageManager) this.storageManager).isReplicated()) {
|
||||
// ARTEMIS-1220: We cannot reuse the same buffer if it's replicated
|
||||
// otherwise there could be another thread still using the buffer on a
|
||||
// replication.
|
||||
bufferToWrite = bufferBytes;
|
||||
} else {
|
||||
bufferToWrite = new byte[bytesRead];
|
||||
System.arraycopy(bufferBytes, 0, bufferToWrite, 0, bytesRead);
|
||||
}
|
||||
|
||||
newMessage.addBytes(bufferToWrite);
|
||||
|
||||
if (bytesRead < bufferBytes.length) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
cloneFile.close();
|
||||
}
|
||||
}
|
||||
|
||||
public SequentialFile createFile() {
|
||||
return storageManager.createFileForLargeMessage(getMessageID(), message.toMessage().isDurable());
|
||||
}
|
||||
|
||||
protected void openFile() throws Exception {
|
||||
if (file == null) {
|
||||
validateFile();
|
||||
} else if (!file.isOpen()) {
|
||||
file.open();
|
||||
}
|
||||
}
|
||||
|
||||
class LargeBodyReaderImpl implements LargeBodyReader {
|
||||
|
||||
private SequentialFile cFile;
|
||||
|
||||
@Override
|
||||
public void open() throws ActiveMQException {
|
||||
try {
|
||||
if (cFile != null && cFile.isOpen()) {
|
||||
cFile.close(false);
|
||||
}
|
||||
cFile = getReadingFile();
|
||||
cFile.open();
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQException(ActiveMQExceptionType.INTERNAL_ERROR, e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void position(long position) throws ActiveMQException {
|
||||
try {
|
||||
cFile.position(position);
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQException(ActiveMQExceptionType.INTERNAL_ERROR, e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long position() {
|
||||
return cFile.position();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws ActiveMQException {
|
||||
try {
|
||||
if (cFile != null) {
|
||||
cFile.close(false);
|
||||
cFile = null;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQInternalErrorException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readInto(final ByteBuffer bufferRead) throws ActiveMQException {
|
||||
try {
|
||||
return cFile.read(bufferRead);
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQInternalErrorException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.activemq.artemis.core.message.LargeBodyEncoder#getSize()
|
||||
*/
|
||||
@Override
|
||||
public long getSize() throws ActiveMQException {
|
||||
return getBodySize();
|
||||
}
|
||||
}
|
||||
|
||||
public boolean hasPendingRecord() {
|
||||
return pendingRecordID != NO_PENDING_ID;
|
||||
}
|
||||
|
||||
public void clearPendingRecordID() {
|
||||
setPendingRecordID(NO_PENDING_ID);
|
||||
}
|
||||
|
||||
|
||||
public long getPendingRecordID() {
|
||||
return this.pendingRecordID;
|
||||
}
|
||||
|
||||
public void setPendingRecordID(long pendingRecordID) {
|
||||
this.pendingRecordID = pendingRecordID;
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -48,7 +48,7 @@ public class LargeMessageTXFailureCallback implements TransactionFailureCallback
|
||||
|
||||
try {
|
||||
LargeServerMessage serverMessage = journalStorageManager.parseLargeMessage(buff);
|
||||
serverMessage.decrementDelayDeletionCount();
|
||||
serverMessage.toMessage().usageDown();
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.journalError(e);
|
||||
}
|
||||
|
@ -16,36 +16,34 @@
|
||||
*/
|
||||
package org.apache.activemq.artemis.core.persistence.impl.journal;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
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.ActiveMQIOErrorException;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQInternalErrorException;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.RefCountMessageListener;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFile;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.CoreLargeServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.apache.activemq.artemis.utils.DataConstants;
|
||||
import org.apache.activemq.artemis.utils.collections.TypedProperties;
|
||||
import org.jboss.logging.Logger;
|
||||
|
||||
import io.netty.buffer.Unpooled;
|
||||
public final class LargeServerMessageImpl extends CoreMessage implements CoreLargeServerMessage {
|
||||
|
||||
public final class LargeServerMessageImpl extends CoreMessage implements LargeServerMessage {
|
||||
@Override
|
||||
public Message toMessage() {
|
||||
return this;
|
||||
}
|
||||
|
||||
// When a message is stored on the journal, it will contain some header and trail on the journal
|
||||
// we need to take that into consideration if that would fit the Journal TimedBuffer.
|
||||
private static final int ESTIMATE_RECORD_TRAIL = 512;
|
||||
|
||||
private final LargeBody largeBody;
|
||||
|
||||
/** This will check if a regular message needs to be converted as large message */
|
||||
public static Message checkLargeMessage(Message message, StorageManager storageManager) throws Exception {
|
||||
if (message.isLargeMessage()) {
|
||||
@ -59,6 +57,11 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishParse() throws Exception {
|
||||
|
||||
}
|
||||
|
||||
private static Message asLargeMessage(Message message, StorageManager storageManager) throws Exception {
|
||||
ICoreMessage coreMessage = message.toCore();
|
||||
LargeServerMessage lsm = storageManager.createLargeMessage(storageManager.generateID(), coreMessage);
|
||||
@ -66,8 +69,8 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
final int readableBytes = buffer.readableBytes();
|
||||
lsm.addBytes(buffer);
|
||||
lsm.releaseResources(true);
|
||||
lsm.putLongProperty(Message.HDR_LARGE_BODY_SIZE, readableBytes);
|
||||
return lsm;
|
||||
lsm.toMessage().putLongProperty(Message.HDR_LARGE_BODY_SIZE, readableBytes);
|
||||
return lsm.toMessage();
|
||||
}
|
||||
|
||||
// Constants -----------------------------------------------------
|
||||
@ -75,26 +78,20 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
|
||||
// Attributes ----------------------------------------------------
|
||||
|
||||
private final JournalStorageManager storageManager;
|
||||
|
||||
private long pendingRecordID = NO_PENDING_ID;
|
||||
|
||||
private boolean paged;
|
||||
|
||||
// We should only use the NIO implementation on the Journal
|
||||
private SequentialFile file;
|
||||
|
||||
private long bodySize = -1;
|
||||
|
||||
private final AtomicInteger delayDeletionCount = new AtomicInteger(0);
|
||||
private final StorageManager storageManager;
|
||||
|
||||
// We cache this
|
||||
private volatile int memoryEstimate = -1;
|
||||
|
||||
public LargeServerMessageImpl(final JournalStorageManager storageManager) {
|
||||
public LargeServerMessageImpl(final StorageManager storageManager) {
|
||||
largeBody = new LargeBody(this, storageManager);
|
||||
this.storageManager = storageManager;
|
||||
}
|
||||
|
||||
public long getBodySize() throws ActiveMQException {
|
||||
return largeBody.getBodySize();
|
||||
}
|
||||
|
||||
/**
|
||||
* Copy constructor
|
||||
*
|
||||
@ -102,17 +99,28 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
* @param copy
|
||||
* @param fileCopy
|
||||
*/
|
||||
private LargeServerMessageImpl(final LargeServerMessageImpl copy,
|
||||
public LargeServerMessageImpl(final LargeServerMessageImpl copy,
|
||||
TypedProperties properties,
|
||||
final SequentialFile fileCopy,
|
||||
final long newID) {
|
||||
super(copy, properties);
|
||||
storageManager = copy.storageManager;
|
||||
file = fileCopy;
|
||||
bodySize = copy.bodySize;
|
||||
largeBody = new LargeBody(this, storageManager, fileCopy);
|
||||
largeBody.setBodySize(copy.largeBody.getStoredBodySize());
|
||||
setMessageID(newID);
|
||||
}
|
||||
|
||||
public LargeServerMessageImpl(byte type,
|
||||
long id,
|
||||
StorageManager storageManager,
|
||||
final SequentialFile fileCopy) {
|
||||
super();
|
||||
this.storageManager = storageManager;
|
||||
setMessageID(id);
|
||||
setType(type);
|
||||
largeBody = new LargeBody(this, storageManager, fileCopy);
|
||||
}
|
||||
|
||||
private static String toDate(long timestamp) {
|
||||
if (timestamp == 0) {
|
||||
return "0";
|
||||
@ -122,6 +130,11 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageManager getStorageManager() {
|
||||
return storageManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isServerMessage() {
|
||||
return true;
|
||||
@ -129,7 +142,17 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
|
||||
@Override
|
||||
public long getPendingRecordID() {
|
||||
return this.pendingRecordID;
|
||||
return largeBody.getPendingRecordID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearPendingRecordID() {
|
||||
largeBody.clearPendingRecordID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPendingRecord() {
|
||||
return largeBody.hasPendingRecord();
|
||||
}
|
||||
|
||||
/**
|
||||
@ -137,40 +160,22 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
*/
|
||||
@Override
|
||||
public void setPendingRecordID(long pendingRecordID) {
|
||||
this.pendingRecordID = pendingRecordID;
|
||||
largeBody.setPendingRecordID(pendingRecordID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPaged() {
|
||||
paged = true;
|
||||
largeBody.setPaged();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void addBytes(final byte[] bytes) throws Exception {
|
||||
validateFile();
|
||||
|
||||
if (!file.isOpen()) {
|
||||
file.open();
|
||||
}
|
||||
|
||||
storageManager.addBytesToLargeMessage(file, getMessageID(), bytes);
|
||||
|
||||
bodySize += bytes.length;
|
||||
largeBody.addBytes(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void addBytes(final ActiveMQBuffer bytes) throws Exception {
|
||||
validateFile();
|
||||
|
||||
if (!file.isOpen()) {
|
||||
file.open();
|
||||
}
|
||||
|
||||
final int readableBytes = bytes.readableBytes();
|
||||
|
||||
storageManager.addBytesToLargeMessage(file, getMessageID(), bytes);
|
||||
|
||||
bodySize += readableBytes;
|
||||
largeBody.addBytes(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -183,79 +188,19 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
}
|
||||
|
||||
public void decode(final ActiveMQBuffer buffer1) {
|
||||
file = null;
|
||||
|
||||
largeBody.clearFile();
|
||||
super.decodeHeadersAndProperties(buffer1.byteBuf());
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void incrementDelayDeletionCount() {
|
||||
delayDeletionCount.incrementAndGet();
|
||||
try {
|
||||
if (paged) {
|
||||
RefCountMessageListener tmpContext = super.getContext();
|
||||
setContext(null);
|
||||
incrementRefCount();
|
||||
setContext(tmpContext);
|
||||
} else {
|
||||
incrementRefCount();
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.errorIncrementDelayDeletionCount(e);
|
||||
}
|
||||
public LargeBodyReader getLargeBodyReader() {
|
||||
return largeBody.getLargeBodyReader();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public synchronized void decrementDelayDeletionCount() throws Exception {
|
||||
int count = delayDeletionCount.decrementAndGet();
|
||||
|
||||
decrementRefCount();
|
||||
|
||||
if (count == 0) {
|
||||
checkDelete();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public LargeBodyEncoder getBodyEncoder() throws ActiveMQException {
|
||||
validateFile();
|
||||
return new DecodingContext();
|
||||
}
|
||||
|
||||
private void checkDelete() throws Exception {
|
||||
if (getRefCount() <= 0) {
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("Deleting file " + file + " as the usage was complete");
|
||||
}
|
||||
|
||||
try {
|
||||
deleteFile();
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.error(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int decrementRefCount() throws Exception {
|
||||
int currentRefCount;
|
||||
if (paged) {
|
||||
RefCountMessageListener tmpContext = super.getContext();
|
||||
setContext(null);
|
||||
currentRefCount = super.decrementRefCount();
|
||||
setContext(tmpContext);
|
||||
} else {
|
||||
currentRefCount = super.decrementRefCount();
|
||||
}
|
||||
|
||||
// We use <= as this could be used by load.
|
||||
// because of a failure, no references were loaded, so we have 0... and we still need to delete the associated
|
||||
// files
|
||||
if (delayDeletionCount.get() <= 0) {
|
||||
checkDelete();
|
||||
}
|
||||
return currentRefCount;
|
||||
protected void releaseComplete() {
|
||||
largeBody.deleteFile();
|
||||
}
|
||||
|
||||
// Even though not recommended, in certain instances
|
||||
@ -263,46 +208,13 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
// in a way you can convert
|
||||
@Override
|
||||
public ActiveMQBuffer getReadOnlyBodyBuffer() {
|
||||
try {
|
||||
validateFile();
|
||||
file.open();
|
||||
int fileSize = (int) file.size();
|
||||
ByteBuffer buffer = ByteBuffer.allocate(fileSize);
|
||||
file.read(buffer);
|
||||
return new ChannelBufferWrapper(Unpooled.wrappedBuffer(buffer));
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
} finally {
|
||||
try {
|
||||
file.close(false);
|
||||
} catch (Exception ignored) {
|
||||
}
|
||||
}
|
||||
|
||||
return largeBody.getReadOnlyBodyBuffer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getBodyBufferSize() {
|
||||
final boolean closeFile = file == null || !file.isOpen();
|
||||
try {
|
||||
openFile();
|
||||
final long fileSize = file.size();
|
||||
int fileSizeAsInt = (int) fileSize;
|
||||
if (fileSizeAsInt < 0) {
|
||||
logger.warnf("suspicious large message file size of %d bytes for %s, will use %d instead.",
|
||||
fileSize, file.getFileName(), Integer.MAX_VALUE);
|
||||
fileSizeAsInt = Integer.MAX_VALUE;
|
||||
}
|
||||
return fileSizeAsInt;
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
} finally {
|
||||
if (closeFile) {
|
||||
try {
|
||||
file.close(false);
|
||||
} catch (Exception ignored) {
|
||||
}
|
||||
}
|
||||
}
|
||||
return largeBody.getBodyBufferSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -312,9 +224,7 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
|
||||
@Override
|
||||
public synchronized void deleteFile() throws Exception {
|
||||
validateFile();
|
||||
releaseResources(false);
|
||||
storageManager.deleteLargeMessageFile(this);
|
||||
largeBody.deleteFile();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -329,16 +239,7 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
|
||||
@Override
|
||||
public synchronized void releaseResources(boolean sync) {
|
||||
if (file != null && file.isOpen()) {
|
||||
try {
|
||||
if (sync) {
|
||||
file.sync();
|
||||
}
|
||||
file.close(false);
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.largeMessageErrorReleasingResources(e);
|
||||
}
|
||||
}
|
||||
largeBody.releaseResources(sync);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -347,11 +248,7 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
super.referenceOriginalMessage(original, originalQueue);
|
||||
|
||||
if (original instanceof LargeServerMessageImpl) {
|
||||
LargeServerMessageImpl otherLM = (LargeServerMessageImpl) original;
|
||||
this.paged = otherLM.paged;
|
||||
if (this.paged) {
|
||||
this.removeAnnotation(Message.HDR_ORIG_MESSAGE_ID);
|
||||
}
|
||||
this.largeBody.referenceOriginalMessage(((LargeServerMessageImpl) original).largeBody);
|
||||
}
|
||||
}
|
||||
|
||||
@ -363,59 +260,19 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
return newMessage;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LargeBody getLargeBody() {
|
||||
return largeBody;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message copy(final long newID) {
|
||||
try {
|
||||
LargeServerMessage newMessage = storageManager.createLargeMessage(newID, this);
|
||||
|
||||
//clone a SequentialFile to avoid concurrent access
|
||||
ensureFileExists(false);
|
||||
SequentialFile cloneFile = file.cloneFile();
|
||||
|
||||
try {
|
||||
byte[] bufferBytes = new byte[100 * 1024];
|
||||
|
||||
ByteBuffer buffer = ByteBuffer.wrap(bufferBytes);
|
||||
|
||||
if (!cloneFile.isOpen()) {
|
||||
cloneFile.open();
|
||||
}
|
||||
|
||||
cloneFile.position(0);
|
||||
|
||||
for (;;) {
|
||||
// The buffer is reused...
|
||||
// We need to make sure we clear the limits and the buffer before reusing it
|
||||
buffer.clear();
|
||||
int bytesRead = cloneFile.read(buffer);
|
||||
|
||||
byte[] bufferToWrite;
|
||||
if (bytesRead <= 0) {
|
||||
break;
|
||||
} else if (bytesRead == bufferBytes.length && !this.storageManager.isReplicated()) {
|
||||
// ARTEMIS-1220: We cannot reuse the same buffer if it's replicated
|
||||
// otherwise there could be another thread still using the buffer on a
|
||||
// replication.
|
||||
bufferToWrite = bufferBytes;
|
||||
} else {
|
||||
bufferToWrite = new byte[bytesRead];
|
||||
System.arraycopy(bufferBytes, 0, bufferToWrite, 0, bytesRead);
|
||||
}
|
||||
|
||||
newMessage.addBytes(bufferToWrite);
|
||||
|
||||
if (bytesRead < bufferBytes.length) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (!file.isOpen()) {
|
||||
newMessage.getFile().close();
|
||||
}
|
||||
cloneFile.close();
|
||||
}
|
||||
|
||||
return newMessage;
|
||||
largeBody.copyInto(newMessage);
|
||||
newMessage.finishParse();
|
||||
newMessage.releaseResources(true);
|
||||
return newMessage.toMessage();
|
||||
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.lareMessageErrorCopying(e, this);
|
||||
@ -424,29 +281,8 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
}
|
||||
|
||||
@Override
|
||||
public SequentialFile getFile() throws ActiveMQException {
|
||||
validateFile();
|
||||
return file;
|
||||
}
|
||||
|
||||
private long getBodySize() throws ActiveMQException {
|
||||
|
||||
try {
|
||||
if (bodySize < 0) {
|
||||
if (file != null) {
|
||||
bodySize = file.size();
|
||||
} else {
|
||||
SequentialFile tmpFile = createFile();
|
||||
bodySize = tmpFile.size();
|
||||
tmpFile.close(false);
|
||||
}
|
||||
}
|
||||
return bodySize;
|
||||
} catch (Exception e) {
|
||||
ActiveMQIOErrorException errorException = new ActiveMQIOErrorException();
|
||||
errorException.initCause(e);
|
||||
throw errorException;
|
||||
}
|
||||
public SequentialFile getAppendFile() throws ActiveMQException {
|
||||
return largeBody.getAppendFile();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -473,93 +309,8 @@ public final class LargeServerMessageImpl extends CoreMessage implements LargeSe
|
||||
}
|
||||
|
||||
public synchronized void ensureFileExists(boolean toOpen) throws ActiveMQException {
|
||||
try {
|
||||
if (file == null) {
|
||||
if (messageID <= 0) {
|
||||
throw new RuntimeException("MessageID not set on LargeMessage");
|
||||
}
|
||||
|
||||
file = createFile();
|
||||
|
||||
if (toOpen) {
|
||||
openFile();
|
||||
}
|
||||
|
||||
bodySize = file.size();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// TODO: There is an IO_ERROR on trunk now, this should be used here instead
|
||||
throw new ActiveMQInternalErrorException(e.getMessage(), e);
|
||||
}
|
||||
largeBody.ensureFileExists(toOpen);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
protected SequentialFile createFile() {
|
||||
return storageManager.createFileForLargeMessage(getMessageID(), durable);
|
||||
}
|
||||
|
||||
protected void openFile() throws Exception {
|
||||
if (file == null) {
|
||||
validateFile();
|
||||
} else if (!file.isOpen()) {
|
||||
file.open();
|
||||
}
|
||||
}
|
||||
|
||||
protected void closeFile() throws Exception {
|
||||
if (file != null && file.isOpen()) {
|
||||
file.close();
|
||||
}
|
||||
}
|
||||
|
||||
// Inner classes -------------------------------------------------
|
||||
|
||||
class DecodingContext implements LargeBodyEncoder {
|
||||
|
||||
private SequentialFile cFile;
|
||||
|
||||
@Override
|
||||
public void open() throws ActiveMQException {
|
||||
try {
|
||||
if (cFile != null && cFile.isOpen()) {
|
||||
cFile.close(false);
|
||||
}
|
||||
cFile = file.cloneFile();
|
||||
cFile.open();
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQException(ActiveMQExceptionType.INTERNAL_ERROR, e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws ActiveMQException {
|
||||
try {
|
||||
if (cFile != null) {
|
||||
cFile.close(false);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQInternalErrorException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int encode(final ByteBuffer bufferRead) throws ActiveMQException {
|
||||
try {
|
||||
return cFile.read(bufferRead);
|
||||
} catch (Exception e) {
|
||||
throw new ActiveMQInternalErrorException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.activemq.artemis.core.message.LargeBodyEncoder#getLargeBodySize()
|
||||
*/
|
||||
@Override
|
||||
public long getLargeBodySize() throws ActiveMQException {
|
||||
return getBodySize();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -50,7 +50,7 @@ public final class LargeServerMessageInSync implements ReplicatedLargeMessage {
|
||||
public synchronized void joinSyncedData(ByteBuffer buffer) throws Exception {
|
||||
if (deleted)
|
||||
return;
|
||||
SequentialFile mainSeqFile = mainLM.getFile();
|
||||
SequentialFile mainSeqFile = mainLM.getAppendFile();
|
||||
if (!mainSeqFile.isOpen()) {
|
||||
mainSeqFile.open();
|
||||
}
|
||||
@ -80,19 +80,19 @@ public final class LargeServerMessageInSync implements ReplicatedLargeMessage {
|
||||
}
|
||||
|
||||
public SequentialFile getSyncFile() throws ActiveMQException {
|
||||
return mainLM.getFile();
|
||||
return mainLM.getAppendFile();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message setDurable(boolean durable) {
|
||||
mainLM.setDurable(durable);
|
||||
return mainLM;
|
||||
return mainLM.toMessage();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Message setMessageID(long id) {
|
||||
mainLM.setMessageID(id);
|
||||
return mainLM;
|
||||
return mainLM.toMessage();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -158,6 +158,16 @@ public final class LargeServerMessageInSync implements ReplicatedLargeMessage {
|
||||
storageManager.addBytesToLargeMessage(appendFile, mainLM.getMessageID(), bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearPendingRecordID() {
|
||||
mainLM.clearPendingRecordID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPendingRecord() {
|
||||
return mainLM.hasPendingRecord();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPendingRecordID(long pendingRecordID) {
|
||||
mainLM.setPendingRecordID(pendingRecordID);
|
||||
|
@ -40,7 +40,7 @@ public class DeleteEncoding implements EncodingSupport {
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.activemq.artemis.core.journal.EncodingSupport#encode(org.apache.activemq.artemis.api.core.ActiveMQBuffer)
|
||||
* @see org.apache.activemq.artemis.core.journal.EncodingSupport#readInto(org.apache.activemq.artemis.api.core.ActiveMQBuffer)
|
||||
*/
|
||||
@Override
|
||||
public void encode(ActiveMQBuffer buffer) {
|
||||
|
@ -19,13 +19,22 @@ package org.apache.activemq.artemis.core.persistence.impl.journal.codec;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
|
||||
public class LargeMessagePersister implements Persister<LargeServerMessage, LargeServerMessage> {
|
||||
import static org.apache.activemq.artemis.core.persistence.PersisterIDs.CoreLargeMessagePersister_ID;
|
||||
|
||||
public class LargeMessagePersister implements Persister<LargeServerMessage> {
|
||||
|
||||
public static final byte ID = CoreLargeMessagePersister_ID;
|
||||
|
||||
private static final LargeMessagePersister theInstance = new LargeMessagePersister();
|
||||
|
||||
@Override
|
||||
public byte getID() {
|
||||
return ID;
|
||||
}
|
||||
|
||||
public static LargeMessagePersister getInstance() {
|
||||
return theInstance;
|
||||
@ -38,7 +47,7 @@ public class LargeMessagePersister implements Persister<LargeServerMessage, Larg
|
||||
* @see org.apache.activemq.artemis.core.journal.EncodingSupport#decode(org.apache.activemq.artemis.spi.core.remoting.ActiveMQBuffer)
|
||||
*/
|
||||
@Override
|
||||
public LargeServerMessage decode(final ActiveMQBuffer buffer, LargeServerMessage message) {
|
||||
public LargeServerMessage decode(final ActiveMQBuffer buffer, LargeServerMessage message, CoreMessageObjectPools objectPools) {
|
||||
((CoreMessage)message).decodeHeadersAndProperties(buffer.byteBuf());
|
||||
return message;
|
||||
}
|
||||
@ -56,7 +65,7 @@ public class LargeMessagePersister implements Persister<LargeServerMessage, Larg
|
||||
*/
|
||||
@Override
|
||||
public int getEncodeSize(LargeServerMessage message) {
|
||||
return message.getEncodeSize();
|
||||
return message.toMessage().getEncodeSize();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -40,7 +40,7 @@ public class PendingLargeMessageEncoding implements EncodingSupport {
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.activemq.artemis.core.journal.EncodingSupport#encode(org.apache.activemq.artemis.spi.core.remoting.ActiveMQBuffer)
|
||||
* @see org.apache.activemq.artemis.core.journal.EncodingSupport#readInto(org.apache.activemq.artemis.spi.core.remoting.ActiveMQBuffer)
|
||||
*/
|
||||
@Override
|
||||
public void encode(final ActiveMQBuffer buffer) {
|
||||
|
@ -22,9 +22,11 @@ import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.buffers.impl.ChannelBufferWrapper;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFile;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.journal.LargeBody;
|
||||
import org.apache.activemq.artemis.core.server.CoreLargeServerMessage;
|
||||
|
||||
class NullStorageLargeServerMessage extends CoreMessage implements LargeServerMessage {
|
||||
class NullStorageLargeServerMessage extends CoreMessage implements CoreLargeServerMessage {
|
||||
|
||||
NullStorageLargeServerMessage() {
|
||||
super();
|
||||
@ -38,6 +40,21 @@ class NullStorageLargeServerMessage extends CoreMessage implements LargeServerMe
|
||||
public void releaseResources(boolean sync) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public LargeBody getLargeBody() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageManager getStorageManager() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message toMessage() {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void addBytes(final byte[] bytes) {
|
||||
if (buffer == null) {
|
||||
@ -48,6 +65,11 @@ class NullStorageLargeServerMessage extends CoreMessage implements LargeServerMe
|
||||
buffer.writeBytes(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishParse() throws Exception {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void addBytes(ActiveMQBuffer bytes) {
|
||||
final int readableBytes = bytes.readableBytes();
|
||||
@ -83,15 +105,6 @@ class NullStorageLargeServerMessage extends CoreMessage implements LargeServerMe
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void decrementDelayDeletionCount() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrementDelayDeletionCount() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isServerMessage() {
|
||||
@ -123,13 +136,23 @@ class NullStorageLargeServerMessage extends CoreMessage implements LargeServerMe
|
||||
public void setPendingRecordID(long pendingRecordID) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearPendingRecordID() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPendingRecord() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPendingRecordID() {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SequentialFile getFile() {
|
||||
public SequentialFile getAppendFile() {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -26,6 +26,8 @@ import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
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.Pair;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
@ -550,6 +552,10 @@ public class NullStorageManager implements StorageManager {
|
||||
// no-op
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteLargeMessageBody(LargeServerMessage largeServerMessage) throws ActiveMQException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean addToPage(PagingStore store,
|
||||
@ -584,6 +590,11 @@ public class NullStorageManager implements StorageManager {
|
||||
// no-op
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBytesToLargeMessage(SequentialFile file, long messageId, ActiveMQBuffer bytes) throws Exception {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void beforePageRead() throws Exception {
|
||||
}
|
||||
|
@ -159,7 +159,20 @@ public interface PostOffice extends ActiveMQComponent {
|
||||
boolean rejectDuplicates,
|
||||
Binding binding) throws Exception;
|
||||
|
||||
MessageReference reroute(Message message, Queue queue, Transaction tx) throws Exception;
|
||||
/**
|
||||
* This method was renamed as reload, use the new method instead
|
||||
* @param message
|
||||
* @param queue
|
||||
* @param tx
|
||||
* @return
|
||||
* @throws Exception
|
||||
*/
|
||||
@Deprecated
|
||||
default MessageReference reroute(Message message, Queue queue, Transaction tx) throws Exception {
|
||||
return reload(message, queue, tx);
|
||||
}
|
||||
|
||||
MessageReference reload(Message message, Queue queue, Transaction tx) throws Exception;
|
||||
|
||||
Pair<RoutingContext, Message> redistribute(Message message,
|
||||
Queue originatingQueue,
|
||||
|
@ -1042,8 +1042,6 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
|
||||
final SimpleString address = context.getAddress(message);
|
||||
|
||||
setPagingStore(address, message);
|
||||
|
||||
AtomicBoolean startedTX = new AtomicBoolean(false);
|
||||
|
||||
applyExpiryDelay(message, address);
|
||||
@ -1175,9 +1173,7 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
}
|
||||
|
||||
@Override
|
||||
public MessageReference reroute(final Message message, final Queue queue, final Transaction tx) throws Exception {
|
||||
|
||||
setPagingStore(queue.getAddress(), message);
|
||||
public MessageReference reload(final Message message, final Queue queue, final Transaction tx) throws Exception {
|
||||
|
||||
MessageReference reference = MessageReference.Factory.createReference(message, queue);
|
||||
|
||||
@ -1189,9 +1185,7 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
}
|
||||
}
|
||||
|
||||
message.incrementDurableRefCount();
|
||||
|
||||
message.incrementRefCount();
|
||||
queue.durableUp(message);
|
||||
|
||||
if (tx == null) {
|
||||
queue.reload(reference);
|
||||
@ -1229,7 +1223,7 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
try {
|
||||
//this will cause large message file to be
|
||||
//cleaned up
|
||||
copyRedistribute.decrementRefCount();
|
||||
// copyRedistribute.refDown();
|
||||
} catch (Exception e) {
|
||||
logger.warn("Failed to clean up message: " + copyRedistribute);
|
||||
}
|
||||
@ -1387,12 +1381,6 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
|
||||
// Private -----------------------------------------------------------------
|
||||
|
||||
private void setPagingStore(SimpleString address, Message message) throws Exception {
|
||||
PagingStore store = pagingManager.getPageStore(CompositeAddress.extractAddressName(address));
|
||||
|
||||
message.setContext(store);
|
||||
}
|
||||
|
||||
private void routeQueueInfo(final Message message, final Queue queue, final boolean applyFilters) throws Exception {
|
||||
if (!applyFilters || queue.getFilter() == null || queue.getFilter().match(message)) {
|
||||
RoutingContext context = new RoutingContextImpl(null);
|
||||
@ -1462,7 +1450,7 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
}
|
||||
refs.add(reference);
|
||||
|
||||
message.incrementRefCount();
|
||||
queue.refUp(message);
|
||||
}
|
||||
|
||||
Iterator<Queue> iter = entry.getValue().getDurableQueues().iterator();
|
||||
@ -1485,7 +1473,7 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
refs.add(reference);
|
||||
|
||||
if (message.isDurable()) {
|
||||
int durableRefCount = message.incrementDurableRefCount();
|
||||
int durableRefCount = queue.durableUp(message);
|
||||
|
||||
if (durableRefCount == 1) {
|
||||
if (tx != null) {
|
||||
@ -1514,9 +1502,9 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
storageManager.updateScheduledDeliveryTime(reference);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
queue.refUp(message);
|
||||
}
|
||||
|
||||
message.incrementRefCount();
|
||||
}
|
||||
}
|
||||
|
||||
@ -1623,7 +1611,7 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
if (!cacheBridge.atomicVerify(bridgeDupBytes, context.getTransaction())) {
|
||||
context.getTransaction().rollback();
|
||||
startedTX.set(false);
|
||||
message.decrementRefCount();
|
||||
message.usageDown(); // this will cause large message delete
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
@ -1649,7 +1637,7 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
context.getTransaction().markAsRollbackOnly(new ActiveMQDuplicateIdException(warnMessage));
|
||||
}
|
||||
|
||||
message.decrementRefCount();
|
||||
message.usageDown(); // this will cause large message delete
|
||||
|
||||
return false;
|
||||
}
|
||||
@ -1836,10 +1824,10 @@ public class PostOfficeImpl implements PostOffice, NotificationListener, Binding
|
||||
Message message = ref.getMessage();
|
||||
|
||||
if (message.isDurable() && ref.getQueue().isDurable()) {
|
||||
message.decrementDurableRefCount();
|
||||
ref.getQueue().durableDown(message);
|
||||
} else {
|
||||
ref.getQueue().refDown(message);
|
||||
}
|
||||
|
||||
message.decrementRefCount();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -26,6 +26,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQIOErrorException;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQInternalErrorException;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQQueueMaxConsumerLimitReached;
|
||||
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;
|
||||
@ -699,7 +700,7 @@ public class ServerSessionPacketHandler implements ChannelHandler {
|
||||
try {
|
||||
final SessionSendMessage message = (SessionSendMessage) packet;
|
||||
requiresResponse = message.isRequiresResponse();
|
||||
this.session.send(EmbedMessageUtil.extractEmbedded(message.getMessage()), this.direct);
|
||||
this.session.send(EmbedMessageUtil.extractEmbedded(message.getMessage(), storageManager), this.direct);
|
||||
if (requiresResponse) {
|
||||
response = createNullResponseMessage(packet);
|
||||
}
|
||||
@ -1026,12 +1027,12 @@ public class ServerSessionPacketHandler implements ChannelHandler {
|
||||
currentLargeMessage.releaseResources(true);
|
||||
|
||||
if (messageBodySize >= 0) {
|
||||
currentLargeMessage.putLongProperty(Message.HDR_LARGE_BODY_SIZE, messageBodySize);
|
||||
currentLargeMessage.toMessage().putLongProperty(Message.HDR_LARGE_BODY_SIZE, messageBodySize);
|
||||
}
|
||||
|
||||
LargeServerMessage message = currentLargeMessage;
|
||||
currentLargeMessage = null;
|
||||
session.doSend(session.getCurrentTransaction(), message, null, false, false);
|
||||
session.doSend(session.getCurrentTransaction(), EmbedMessageUtil.extractEmbedded((ICoreMessage)message.toMessage(), storageManager), null, false, false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -23,7 +23,6 @@ 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.api.core.client.ActiveMQClient;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessagePersister;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
@ -38,7 +37,7 @@ public class CoreProtocolManagerFactory extends AbstractProtocolManagerFactory<I
|
||||
private static final String MODULE_NAME = "artemis-server";
|
||||
|
||||
@Override
|
||||
public Persister<Message, CoreMessageObjectPools>[] getPersister() {
|
||||
public Persister<Message>[] getPersister() {
|
||||
return new Persister[]{CoreMessagePersister.getInstance()};
|
||||
}
|
||||
|
||||
|
@ -18,7 +18,7 @@ package org.apache.activemq.artemis.core.protocol.core.impl;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.protocol.core.Channel;
|
||||
import org.apache.activemq.artemis.core.protocol.core.Packet;
|
||||
import org.apache.activemq.artemis.core.protocol.core.ServerSessionPacketHandler;
|
||||
|
@ -52,8 +52,12 @@ public interface ReplicatedLargeMessage {
|
||||
*/
|
||||
void addBytes(byte[] body) throws Exception;
|
||||
|
||||
void clearPendingRecordID();
|
||||
|
||||
void setPendingRecordID(long pendingRecordID);
|
||||
|
||||
long getPendingRecordID();
|
||||
|
||||
boolean hasPendingRecord();
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,26 @@
|
||||
/*
|
||||
* 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.server;
|
||||
|
||||
/**
|
||||
* This is a tagging interface,
|
||||
* as we need to make sure the LargeMessage is a core large message is certain places.
|
||||
*/
|
||||
public interface CoreLargeServerMessage extends LargeServerMessage {
|
||||
|
||||
}
|
@ -18,19 +18,26 @@ package org.apache.activemq.artemis.core.server;
|
||||
|
||||
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.Message;
|
||||
import org.apache.activemq.artemis.core.io.SequentialFile;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.journal.LargeBody;
|
||||
import org.apache.activemq.artemis.core.replication.ReplicatedLargeMessage;
|
||||
|
||||
public interface LargeServerMessage extends ReplicatedLargeMessage, ICoreMessage {
|
||||
public interface LargeServerMessage extends ReplicatedLargeMessage {
|
||||
|
||||
long NO_PENDING_ID = -1;
|
||||
Message toMessage();
|
||||
|
||||
StorageManager getStorageManager();
|
||||
|
||||
@Override
|
||||
void addBytes(byte[] bytes) throws Exception;
|
||||
|
||||
void addBytes(ActiveMQBuffer bytes) throws Exception;
|
||||
|
||||
long getMessageID();
|
||||
|
||||
/**
|
||||
* We have to copy the large message content in case of DLQ and paged messages
|
||||
* For that we need to pre-mark the LargeMessage with a flag when it is paged
|
||||
@ -46,13 +53,16 @@ public interface LargeServerMessage extends ReplicatedLargeMessage, ICoreMessage
|
||||
@Override
|
||||
void deleteFile() throws Exception;
|
||||
|
||||
void incrementDelayDeletionCount();
|
||||
|
||||
void decrementDelayDeletionCount() throws Exception;
|
||||
|
||||
/**
|
||||
* This will return the File suitable for appending the message
|
||||
* @return
|
||||
* @throws ActiveMQException
|
||||
*/
|
||||
SequentialFile getFile() throws ActiveMQException;
|
||||
SequentialFile getAppendFile() throws ActiveMQException;
|
||||
|
||||
LargeBodyReader getLargeBodyReader() throws ActiveMQException;
|
||||
|
||||
LargeBody getLargeBody();
|
||||
|
||||
void finishParse() throws Exception;
|
||||
}
|
||||
|
@ -27,6 +27,7 @@ import org.apache.activemq.artemis.api.core.Pair;
|
||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||
import org.apache.activemq.artemis.api.core.SimpleString;
|
||||
import org.apache.activemq.artemis.core.filter.Filter;
|
||||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
|
||||
import org.apache.activemq.artemis.core.persistence.OperationContext;
|
||||
import org.apache.activemq.artemis.core.postoffice.Binding;
|
||||
@ -48,6 +49,8 @@ public interface Queue extends Bindable,CriticalComponent {
|
||||
|
||||
void setFilter(Filter filter);
|
||||
|
||||
PagingStore getPagingStore();
|
||||
|
||||
PageSubscription getPageSubscription();
|
||||
|
||||
RoutingType getRoutingType();
|
||||
@ -60,6 +63,15 @@ public interface Queue extends Bindable,CriticalComponent {
|
||||
|
||||
boolean isDurable();
|
||||
|
||||
int durableUp(Message message);
|
||||
|
||||
int durableDown(Message message);
|
||||
|
||||
void refUp(Message message);
|
||||
|
||||
void refDown(Message message);
|
||||
|
||||
|
||||
/**
|
||||
* The queue definition could be durable, but the messages could eventually be considered non durable.
|
||||
* (e.g. purgeOnNoConsumers)
|
||||
|
@ -31,6 +31,7 @@ public final class QueueConfig {
|
||||
private final SimpleString address;
|
||||
private final SimpleString name;
|
||||
private final Filter filter;
|
||||
final PagingStore pagingStore;
|
||||
private final PageSubscription pageSubscription;
|
||||
private final SimpleString user;
|
||||
private final boolean durable;
|
||||
@ -259,10 +260,11 @@ public final class QueueConfig {
|
||||
* @throws IllegalStateException if the creation of {@link PageSubscription} fails
|
||||
*/
|
||||
public QueueConfig build() {
|
||||
final PagingStore pageStore;
|
||||
final PageSubscription pageSubscription;
|
||||
if (pagingManager != null && !FilterUtils.isTopicIdentification(filter)) {
|
||||
try {
|
||||
final PagingStore pageStore = this.pagingManager.getPageStore(address);
|
||||
pageStore = this.pagingManager.getPageStore(address);
|
||||
if (pageStore != null) {
|
||||
pageSubscription = pageStore.getCursorProvider().createSubscription(id, filter, durable);
|
||||
} else {
|
||||
@ -273,8 +275,9 @@ public final class QueueConfig {
|
||||
}
|
||||
} else {
|
||||
pageSubscription = null;
|
||||
pageStore = null;
|
||||
}
|
||||
return new QueueConfig(id, address, name, filter, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, lastValue, lastValueKey, nonDestructive, consumersBeforeDispatch, delayBeforeDispatch, purgeOnNoConsumers, groupRebalance, groupBuckets, groupFirstKey, autoDelete, autoDeleteDelay, autoDeleteMessageCount, ringSize, configurationManaged);
|
||||
return new QueueConfig(id, address, name, filter, pageStore, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, lastValue, lastValueKey, nonDestructive, consumersBeforeDispatch, delayBeforeDispatch, purgeOnNoConsumers, groupRebalance, groupBuckets, groupFirstKey, autoDelete, autoDeleteDelay, autoDeleteMessageCount, ringSize, configurationManaged);
|
||||
}
|
||||
|
||||
}
|
||||
@ -311,6 +314,7 @@ public final class QueueConfig {
|
||||
final SimpleString address,
|
||||
final SimpleString name,
|
||||
final Filter filter,
|
||||
final PagingStore pagingStore,
|
||||
final PageSubscription pageSubscription,
|
||||
final SimpleString user,
|
||||
final boolean durable,
|
||||
@ -337,6 +341,7 @@ public final class QueueConfig {
|
||||
this.address = address;
|
||||
this.name = name;
|
||||
this.filter = filter;
|
||||
this.pagingStore = pagingStore;
|
||||
this.pageSubscription = pageSubscription;
|
||||
this.user = user;
|
||||
this.durable = durable;
|
||||
@ -465,6 +470,10 @@ public final class QueueConfig {
|
||||
return ringSize;
|
||||
}
|
||||
|
||||
public PagingStore getPagingStore() {
|
||||
return pagingStore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
|
@ -32,6 +32,7 @@ import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQInterruptedException;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.Pair;
|
||||
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.api.core.TransportConfiguration;
|
||||
@ -552,6 +553,7 @@ public class BridgeImpl implements Bridge, SessionFailureListener, SendAcknowled
|
||||
/* Hook for processing message before forwarding */
|
||||
protected Message beforeForward(Message message, final SimpleString forwardingAddress) {
|
||||
message = message.copy();
|
||||
((RefCountMessage)message).setParentRef((RefCountMessage)message);
|
||||
|
||||
return beforeForwardingNoCopy(message, forwardingAddress);
|
||||
}
|
||||
@ -662,10 +664,10 @@ public class BridgeImpl implements Bridge, SessionFailureListener, SendAcknowled
|
||||
final HandleStatus status;
|
||||
if (message.isLargeMessage()) {
|
||||
deliveringLargeMessage = true;
|
||||
deliverLargeMessage(dest, ref, (LargeServerMessage) message);
|
||||
deliverLargeMessage(dest, ref, (LargeServerMessage) message, ref.getMessage());
|
||||
status = HandleStatus.HANDLED;
|
||||
} else {
|
||||
status = deliverStandardMessage(dest, ref, message);
|
||||
status = deliverStandardMessage(dest, ref, message, ref.getMessage());
|
||||
}
|
||||
|
||||
//Only increment messages pending acknowledgement if handled by bridge
|
||||
@ -764,12 +766,13 @@ public class BridgeImpl implements Bridge, SessionFailureListener, SendAcknowled
|
||||
|
||||
private void deliverLargeMessage(final SimpleString dest,
|
||||
final MessageReference ref,
|
||||
final LargeServerMessage message) {
|
||||
final LargeServerMessage message,
|
||||
final Message originalMessage) {
|
||||
executor.execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
producer.send(dest, message);
|
||||
producer.send(dest, message.toMessage());
|
||||
|
||||
// as soon as we are done sending the large message
|
||||
// we unset the delivery flag and we will call the deliveryAsync on the queue
|
||||
@ -794,7 +797,7 @@ public class BridgeImpl implements Bridge, SessionFailureListener, SendAcknowled
|
||||
* @param message
|
||||
* @return
|
||||
*/
|
||||
private HandleStatus deliverStandardMessage(SimpleString dest, final MessageReference ref, Message message) {
|
||||
private HandleStatus deliverStandardMessage(SimpleString dest, final MessageReference ref, Message message, Message originalMessage) {
|
||||
// if we failover during send then there is a chance that the
|
||||
// that this will throw a disconnect, we need to remove the message
|
||||
// from the acks so it will get resent, duplicate detection will cope
|
||||
@ -821,6 +824,8 @@ public class BridgeImpl implements Bridge, SessionFailureListener, SendAcknowled
|
||||
connectionFailed(e, false);
|
||||
|
||||
return HandleStatus.BUSY;
|
||||
} finally {
|
||||
originalMessage.usageDown();
|
||||
}
|
||||
|
||||
return HandleStatus.HANDLED;
|
||||
|
@ -28,6 +28,7 @@ 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.filter.Filter;
|
||||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.postoffice.PostOffice;
|
||||
@ -60,6 +61,7 @@ public class LastValueQueue extends QueueImpl {
|
||||
final SimpleString address,
|
||||
final SimpleString name,
|
||||
final Filter filter,
|
||||
final PagingStore pagingStore,
|
||||
final PageSubscription pageSubscription,
|
||||
final SimpleString user,
|
||||
final boolean durable,
|
||||
@ -87,7 +89,7 @@ public class LastValueQueue extends QueueImpl {
|
||||
final ArtemisExecutor executor,
|
||||
final ActiveMQServer server,
|
||||
final QueueFactory factory) {
|
||||
super(persistenceID, address, name, filter, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, groupRebalance, groupBuckets, groupFirstKey, nonDestructive, consumersBeforeDispatch, delayBeforeDispatch, purgeOnNoConsumers, autoDelete, autoDeleteDelay, autoDeleteMessageCount, configurationManaged, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
super(persistenceID, address, name, filter, pagingStore, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, groupRebalance, groupBuckets, groupFirstKey, nonDestructive, consumersBeforeDispatch, delayBeforeDispatch, purgeOnNoConsumers, autoDelete, autoDeleteDelay, autoDeleteMessageCount, configurationManaged, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
this.lastValueKey = lastValueKey;
|
||||
}
|
||||
|
||||
|
@ -238,7 +238,7 @@ public class PostOfficeJournalLoader implements JournalLoader {
|
||||
record.getMessage().setScheduledDeliveryTime(scheduledDeliveryTime);
|
||||
}
|
||||
|
||||
MessageReference ref = postOffice.reroute(record.getMessage(), queue, null);
|
||||
MessageReference ref = postOffice.reload(record.getMessage(), queue, null);
|
||||
|
||||
ref.setDeliveryCount(record.getDeliveryCount());
|
||||
|
||||
@ -252,7 +252,7 @@ public class PostOfficeJournalLoader implements JournalLoader {
|
||||
@Override
|
||||
public void handleNoMessageReferences(Map<Long, Message> messages) {
|
||||
for (Message msg : messages.values()) {
|
||||
if (msg.getRefCount() == 0) {
|
||||
if (msg.getRefCount() == 0 && msg.getDurableCount() == 0) {
|
||||
ActiveMQServerLogger.LOGGER.journalUnreferencedMessage(msg.getMessageID());
|
||||
try {
|
||||
storageManager.deleteMessage(msg.getMessageID());
|
||||
@ -308,7 +308,7 @@ public class PostOfficeJournalLoader implements JournalLoader {
|
||||
ActiveMQServerLogger.LOGGER.journalMessageInPreparedTX(queueID);
|
||||
return;
|
||||
}
|
||||
postOffice.reroute(message, queue, tx);
|
||||
postOffice.reload(message, queue, tx);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -75,9 +75,9 @@ public class QueueFactoryImpl implements QueueFactory {
|
||||
public Queue createQueueWith(final QueueConfig config) {
|
||||
final Queue queue;
|
||||
if (lastValueKey(config) != null) {
|
||||
queue = new LastValueQueue(config.id(), config.address(), config.name(), config.filter(), config.pageSubscription(), config.user(), config.isDurable(), config.isTemporary(), config.isAutoCreated(), config.deliveryMode(), config.maxConsumers(), config.isExclusive(), config.isGroupRebalance(), config.getGroupBuckets(), config.getGroupFirstKey(), config.consumersBeforeDispatch(), config.delayBeforeDispatch(), config.isPurgeOnNoConsumers(), lastValueKey(config), config.isNonDestructive(), config.isAutoDelete(), config.getAutoDeleteDelay(), config.getAutoDeleteMessageCount(), config.isConfigurationManaged(), scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executorFactory.getExecutor(), server, this);
|
||||
queue = new LastValueQueue(config.id(), config.address(), config.name(), config.filter(), config.getPagingStore(), config.pageSubscription(), config.user(), config.isDurable(), config.isTemporary(), config.isAutoCreated(), config.deliveryMode(), config.maxConsumers(), config.isExclusive(), config.isGroupRebalance(), config.getGroupBuckets(), config.getGroupFirstKey(), config.consumersBeforeDispatch(), config.delayBeforeDispatch(), config.isPurgeOnNoConsumers(), lastValueKey(config), config.isNonDestructive(), config.isAutoDelete(), config.getAutoDeleteDelay(), config.getAutoDeleteMessageCount(), config.isConfigurationManaged(), scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executorFactory.getExecutor(), server, this);
|
||||
} else {
|
||||
queue = new QueueImpl(config.id(), config.address(), config.name(), config.filter(), config.pageSubscription(), config.user(), config.isDurable(), config.isTemporary(), config.isAutoCreated(), config.deliveryMode(), config.maxConsumers(), config.isExclusive(), config.isGroupRebalance(), config.getGroupBuckets(), config.getGroupFirstKey(), config.isNonDestructive(), config.consumersBeforeDispatch(), config.delayBeforeDispatch(), config.isPurgeOnNoConsumers(), config.isAutoDelete(), config.getAutoDeleteDelay(), config.getAutoDeleteMessageCount(), config.isConfigurationManaged(), config.getRingSize(), scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executorFactory.getExecutor(), server, this);
|
||||
queue = new QueueImpl(config.id(), config.address(), config.name(), config.filter(), config.getPagingStore(), config.pageSubscription(), config.user(), config.isDurable(), config.isTemporary(), config.isAutoCreated(), config.deliveryMode(), config.maxConsumers(), config.isExclusive(), config.isGroupRebalance(), config.getGroupBuckets(), config.getGroupFirstKey(), config.isNonDestructive(), config.consumersBeforeDispatch(), config.delayBeforeDispatch(), config.isPurgeOnNoConsumers(), config.isAutoDelete(), config.getAutoDeleteDelay(), config.getAutoDeleteMessageCount(), config.isConfigurationManaged(), config.getRingSize(), scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executorFactory.getExecutor(), server, this);
|
||||
}
|
||||
server.getCriticalAnalyzer().add(queue);
|
||||
return queue;
|
||||
@ -102,9 +102,9 @@ public class QueueFactoryImpl implements QueueFactory {
|
||||
|
||||
Queue queue;
|
||||
if (lastValueKey(addressSettings) != null) {
|
||||
queue = new LastValueQueue(persistenceID, address, name, filter, pageSubscription, user, durable, temporary, autoCreated, ActiveMQDefaultConfiguration.getDefaultRoutingType(), ActiveMQDefaultConfiguration.getDefaultMaxQueueConsumers(), ActiveMQDefaultConfiguration.getDefaultExclusive(), ActiveMQDefaultConfiguration.getDefaultGroupRebalance(), ActiveMQDefaultConfiguration.getDefaultGroupBuckets(), ActiveMQDefaultConfiguration.getDefaultGroupFirstKey(), ActiveMQDefaultConfiguration.getDefaultConsumersBeforeDispatch(), ActiveMQDefaultConfiguration.getDefaultDelayBeforeDispatch(), ActiveMQDefaultConfiguration.getDefaultPurgeOnNoConsumers(), lastValueKey(addressSettings), ActiveMQDefaultConfiguration.getDefaultNonDestructive(), ActiveMQDefaultConfiguration.getDefaultQueueAutoDelete(autoCreated), ActiveMQDefaultConfiguration.getDefaultQueueAutoDeleteDelay(), ActiveMQDefaultConfiguration.getDefaultQueueAutoDeleteMessageCount(),false, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executorFactory.getExecutor(), server, this);
|
||||
queue = new LastValueQueue(persistenceID, address, name, filter, pageSubscription == null ? null : pageSubscription.getPagingStore(), pageSubscription, user, durable, temporary, autoCreated, ActiveMQDefaultConfiguration.getDefaultRoutingType(), ActiveMQDefaultConfiguration.getDefaultMaxQueueConsumers(), ActiveMQDefaultConfiguration.getDefaultExclusive(), ActiveMQDefaultConfiguration.getDefaultGroupRebalance(), ActiveMQDefaultConfiguration.getDefaultGroupBuckets(), ActiveMQDefaultConfiguration.getDefaultGroupFirstKey(), ActiveMQDefaultConfiguration.getDefaultConsumersBeforeDispatch(), ActiveMQDefaultConfiguration.getDefaultDelayBeforeDispatch(), ActiveMQDefaultConfiguration.getDefaultPurgeOnNoConsumers(), lastValueKey(addressSettings), ActiveMQDefaultConfiguration.getDefaultNonDestructive(), ActiveMQDefaultConfiguration.getDefaultQueueAutoDelete(autoCreated), ActiveMQDefaultConfiguration.getDefaultQueueAutoDeleteDelay(), ActiveMQDefaultConfiguration.getDefaultQueueAutoDeleteMessageCount(),false, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executorFactory.getExecutor(), server, this);
|
||||
} else {
|
||||
queue = new QueueImpl(persistenceID, address, name, filter, pageSubscription, user, durable, temporary, autoCreated, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executorFactory.getExecutor(), server, this);
|
||||
queue = new QueueImpl(persistenceID, address, name, filter, pageSubscription == null ? null : pageSubscription.getPagingStore(), pageSubscription, user, durable, temporary, autoCreated, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executorFactory.getExecutor(), server, this);
|
||||
}
|
||||
|
||||
server.getCriticalAnalyzer().add(queue);
|
||||
|
@ -56,6 +56,7 @@ import org.apache.activemq.artemis.api.core.management.ResourceNames;
|
||||
import org.apache.activemq.artemis.core.PriorityAware;
|
||||
import org.apache.activemq.artemis.core.filter.Filter;
|
||||
import org.apache.activemq.artemis.core.io.IOCallback;
|
||||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.PageIterator;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.PagePosition;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
|
||||
@ -168,6 +169,8 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
|
||||
private volatile boolean queueDestroyed = false;
|
||||
|
||||
private final PagingStore pagingStore;
|
||||
|
||||
private final PageSubscription pageSubscription;
|
||||
|
||||
private ReferenceCounter refCountForConsumers;
|
||||
@ -370,32 +373,34 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
final ArtemisExecutor executor,
|
||||
final ActiveMQServer server,
|
||||
final QueueFactory factory) {
|
||||
this(id, address, name, filter, null, user, durable, temporary, autoCreated, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
}
|
||||
|
||||
public QueueImpl(final long id,
|
||||
final SimpleString address,
|
||||
final SimpleString name,
|
||||
final Filter filter,
|
||||
final PageSubscription pageSubscription,
|
||||
final SimpleString user,
|
||||
final boolean durable,
|
||||
final boolean temporary,
|
||||
final boolean autoCreated,
|
||||
final ScheduledExecutorService scheduledExecutor,
|
||||
final PostOffice postOffice,
|
||||
final StorageManager storageManager,
|
||||
final HierarchicalRepository<AddressSettings> addressSettingsRepository,
|
||||
final ArtemisExecutor executor,
|
||||
final ActiveMQServer server,
|
||||
final QueueFactory factory) {
|
||||
this(id, address, name, filter, pageSubscription, user, durable, temporary, autoCreated, RoutingType.MULTICAST, null, null, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
this(id, address, name, filter, null, null, user, durable, temporary, autoCreated, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
}
|
||||
|
||||
public QueueImpl(final long id,
|
||||
final SimpleString address,
|
||||
final SimpleString name,
|
||||
final Filter filter,
|
||||
final PagingStore pagingStore,
|
||||
final PageSubscription pageSubscription,
|
||||
final SimpleString user,
|
||||
final boolean durable,
|
||||
final boolean temporary,
|
||||
final boolean autoCreated,
|
||||
final ScheduledExecutorService scheduledExecutor,
|
||||
final PostOffice postOffice,
|
||||
final StorageManager storageManager,
|
||||
final HierarchicalRepository<AddressSettings> addressSettingsRepository,
|
||||
final ArtemisExecutor executor,
|
||||
final ActiveMQServer server,
|
||||
final QueueFactory factory) {
|
||||
this(id, address, name, filter, pagingStore, pageSubscription, user, durable, temporary, autoCreated, RoutingType.MULTICAST, null, null, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
}
|
||||
|
||||
public QueueImpl(final long id,
|
||||
final SimpleString address,
|
||||
final SimpleString name,
|
||||
final Filter filter,
|
||||
final PagingStore pagingStore,
|
||||
final PageSubscription pageSubscription,
|
||||
final SimpleString user,
|
||||
final boolean durable,
|
||||
@ -411,13 +416,14 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
final ArtemisExecutor executor,
|
||||
final ActiveMQServer server,
|
||||
final QueueFactory factory) {
|
||||
this(id, address, name, filter, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, null, purgeOnNoConsumers, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
this(id, address, name, filter, pagingStore, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, null, purgeOnNoConsumers, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
}
|
||||
|
||||
public QueueImpl(final long id,
|
||||
final SimpleString address,
|
||||
final SimpleString name,
|
||||
final Filter filter,
|
||||
final PagingStore pagingStore,
|
||||
final PageSubscription pageSubscription,
|
||||
final SimpleString user,
|
||||
final boolean durable,
|
||||
@ -434,13 +440,14 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
final ArtemisExecutor executor,
|
||||
final ActiveMQServer server,
|
||||
final QueueFactory factory) {
|
||||
this(id, address, name, filter, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, null, null, false, null, null, purgeOnNoConsumers, null, null, null, false, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
this(id, address, name, filter, pagingStore, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, null, null, false, null, null, purgeOnNoConsumers, null, null, null, false, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
}
|
||||
|
||||
public QueueImpl(final long id,
|
||||
final SimpleString address,
|
||||
final SimpleString name,
|
||||
final Filter filter,
|
||||
final PagingStore pagingStore,
|
||||
final PageSubscription pageSubscription,
|
||||
final SimpleString user,
|
||||
final boolean durable,
|
||||
@ -466,13 +473,14 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
final ArtemisExecutor executor,
|
||||
final ActiveMQServer server,
|
||||
final QueueFactory factory) {
|
||||
this(id, address, name, filter, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, groupRebalance, groupBuckets, null, nonDestructive, consumersBeforeDispatch, delayBeforeDispatch, purgeOnNoConsumers, autoDelete, autoDeleteDelay, autoDeleteMessageCount, configurationManaged, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
this(id, address, name, filter, pagingStore, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, groupRebalance, groupBuckets, null, nonDestructive, consumersBeforeDispatch, delayBeforeDispatch, purgeOnNoConsumers, autoDelete, autoDeleteDelay, autoDeleteMessageCount, configurationManaged, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
}
|
||||
|
||||
public QueueImpl(final long id,
|
||||
final SimpleString address,
|
||||
final SimpleString name,
|
||||
final Filter filter,
|
||||
final PagingStore pagingStore,
|
||||
final PageSubscription pageSubscription,
|
||||
final SimpleString user,
|
||||
final boolean durable,
|
||||
@ -499,13 +507,14 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
final ArtemisExecutor executor,
|
||||
final ActiveMQServer server,
|
||||
final QueueFactory factory) {
|
||||
this(id, address, name, filter, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, groupRebalance, groupBuckets, groupFirstKey, nonDestructive, consumersBeforeDispatch, delayBeforeDispatch, purgeOnNoConsumers, autoDelete, autoDeleteDelay, autoDeleteMessageCount, configurationManaged, null, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
this(id, address, name, filter, pagingStore, pageSubscription, user, durable, temporary, autoCreated, routingType, maxConsumers, exclusive, groupRebalance, groupBuckets, groupFirstKey, nonDestructive, consumersBeforeDispatch, delayBeforeDispatch, purgeOnNoConsumers, autoDelete, autoDeleteDelay, autoDeleteMessageCount, configurationManaged, null, scheduledExecutor, postOffice, storageManager, addressSettingsRepository, executor, server, factory);
|
||||
}
|
||||
|
||||
public QueueImpl(final long id,
|
||||
final SimpleString address,
|
||||
final SimpleString name,
|
||||
final Filter filter,
|
||||
final PagingStore pagingStore,
|
||||
final PageSubscription pageSubscription,
|
||||
final SimpleString user,
|
||||
final boolean durable,
|
||||
@ -547,6 +556,8 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
|
||||
this.filter = filter;
|
||||
|
||||
this.pagingStore = pagingStore;
|
||||
|
||||
this.pageSubscription = pageSubscription;
|
||||
|
||||
this.propertyDurable = durable;
|
||||
@ -868,6 +879,47 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
return id;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int durableUp(Message message) {
|
||||
int count = message.durableUp();
|
||||
if (pagingStore != null) {
|
||||
pagingStore.durableUp(message, count);
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int durableDown(Message message) {
|
||||
int count = message.durableDown();
|
||||
if (pagingStore != null) {
|
||||
pagingStore.durableDown(message, count);
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void refUp(Message message) {
|
||||
int count = message.refUp();
|
||||
if (pagingStore != null) {
|
||||
pagingStore.refUp(message, count);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void refDown(Message message) {
|
||||
int count = message.refDown();
|
||||
if (pagingStore != null) {
|
||||
pagingStore.refDown(message, count);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PagingStore getPagingStore() {
|
||||
return pagingStore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PageSubscription getPageSubscription() {
|
||||
return pageSubscription;
|
||||
@ -3654,14 +3706,8 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
|
||||
boolean durableRef = message.isDurable() && queue.isDurable();
|
||||
|
||||
try {
|
||||
message.decrementRefCount();
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.errorDecrementingRefCount(e);
|
||||
}
|
||||
|
||||
if (durableRef) {
|
||||
int count = message.decrementDurableRefCount();
|
||||
int count = queue.durableDown(message);
|
||||
|
||||
if (count == 0) {
|
||||
// Note - we MUST store the delete after the preceding ack has been committed to storage, we cannot combine
|
||||
@ -3684,6 +3730,8 @@ public class QueueImpl extends CriticalComponentImpl implements Queue {
|
||||
ActiveMQServerLogger.LOGGER.errorRemovingMessage(e, message.getMessageID());
|
||||
}
|
||||
}
|
||||
} else {
|
||||
queue.refDown(message);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -23,7 +23,6 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.NonExistentPage;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.PagedReference;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
@ -134,7 +133,7 @@ public class RefsOperation extends TransactionOperationAbstract {
|
||||
for (MessageReference ref : ackedRefs) {
|
||||
Message message = ref.getMessage();
|
||||
if (message.isDurable()) {
|
||||
int durableRefCount = message.incrementDurableRefCount();
|
||||
int durableRefCount = ref.getQueue().durableUp(ref.getMessage());
|
||||
|
||||
if (durableRefCount == 1) {
|
||||
storageManager.storeMessageTransactional(ackedTX.getID(), message);
|
||||
@ -146,7 +145,8 @@ public class RefsOperation extends TransactionOperationAbstract {
|
||||
ackedTX.setContainsPersistent();
|
||||
}
|
||||
|
||||
message.incrementRefCount();
|
||||
// TODO-NOW: THIS MUST BE SOLVED BEFORE MERGED, DO NOT LET ME COMMIT THIS WITHOUT REVIEW
|
||||
queue.refUp(message);
|
||||
}
|
||||
ackedTX.commit(true);
|
||||
} catch (Exception e) {
|
||||
@ -190,7 +190,7 @@ public class RefsOperation extends TransactionOperationAbstract {
|
||||
for (MessageReference refmsg : pagedMessagesToPostACK) {
|
||||
((PagedReference)refmsg).removePendingFlag();
|
||||
if (((PagedReference) refmsg).isLargeMessage()) {
|
||||
decrementRefCount(refmsg);
|
||||
refmsg.getQueue().refDown(refmsg.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -202,17 +202,6 @@ public class RefsOperation extends TransactionOperationAbstract {
|
||||
}
|
||||
}
|
||||
|
||||
private void decrementRefCount(MessageReference refmsg) {
|
||||
try {
|
||||
refmsg.getMessage().decrementRefCount();
|
||||
} catch (NonExistentPage e) {
|
||||
// This could happen on after commit, since the page could be deleted on file earlier by another thread
|
||||
logger.debug(e);
|
||||
} catch (Exception e) {
|
||||
ActiveMQServerLogger.LOGGER.failedToDecrementMessageReferenceCount(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized List<MessageReference> getRelatedMessageReferences() {
|
||||
List<MessageReference> listRet = new LinkedList<>();
|
||||
|
@ -39,7 +39,7 @@ import org.apache.activemq.artemis.api.core.management.CoreNotificationType;
|
||||
import org.apache.activemq.artemis.api.core.management.ManagementHelper;
|
||||
import org.apache.activemq.artemis.core.client.impl.ClientConsumerImpl;
|
||||
import org.apache.activemq.artemis.core.filter.Filter;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyEncoder;
|
||||
import org.apache.activemq.artemis.core.message.LargeBodyReader;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.postoffice.Binding;
|
||||
@ -49,6 +49,7 @@ import org.apache.activemq.artemis.core.protocol.core.impl.PacketImpl;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQMessageBundle;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
|
||||
import org.apache.activemq.artemis.core.server.CoreLargeServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.HandleStatus;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.apache.activemq.artemis.core.server.MessageReference;
|
||||
@ -112,7 +113,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
|
||||
private boolean started;
|
||||
|
||||
private volatile LargeMessageDeliverer largeMessageDeliverer = null;
|
||||
private volatile CoreLargeMessageDeliverer largeMessageDeliverer = null;
|
||||
|
||||
@Override
|
||||
public String debug() {
|
||||
@ -458,10 +459,16 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
}
|
||||
}
|
||||
|
||||
// The deliverer will increase the usageUp, so the preAck has to be done after this is created
|
||||
// otherwise we may have a removed message early on
|
||||
if (message instanceof CoreLargeServerMessage && this.supportLargeMessage) {
|
||||
largeMessageDeliverer = new CoreLargeMessageDeliverer((LargeServerMessage) message, ref);
|
||||
}
|
||||
|
||||
if (preAcknowledge) {
|
||||
if (message.isLargeMessage()) {
|
||||
// we must hold one reference, or the file will be deleted before it could be delivered
|
||||
((LargeServerMessage) message).incrementDelayDeletionCount();
|
||||
((LargeServerMessage) message).toMessage().usageUp();
|
||||
}
|
||||
|
||||
// With pre-ack, we ack *before* sending to the client
|
||||
@ -469,10 +476,6 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
acks++;
|
||||
}
|
||||
|
||||
if (message.isLargeMessage() && this.supportLargeMessage) {
|
||||
largeMessageDeliverer = new LargeMessageDeliverer((LargeServerMessage) message, ref);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
pendingDelivery.countUp();
|
||||
@ -490,12 +493,12 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
server.callBrokerMessagePlugins(plugin -> plugin.beforeDeliver(this, reference));
|
||||
}
|
||||
|
||||
if (message.isLargeMessage() && supportLargeMessage) {
|
||||
if (message instanceof CoreLargeServerMessage && supportLargeMessage) {
|
||||
if (largeMessageDeliverer == null) {
|
||||
// This can't really happen as handle had already crated the deliverer
|
||||
// instead of throwing an exception in weird cases there is no problem on just go ahead and create it
|
||||
// again here
|
||||
largeMessageDeliverer = new LargeMessageDeliverer((LargeServerMessage) message, reference);
|
||||
largeMessageDeliverer = new CoreLargeMessageDeliverer((LargeServerMessage) message, reference);
|
||||
}
|
||||
// The deliverer was prepared during handle, as we can't have more than one pending large message
|
||||
// as it would return busy if there is anything pending
|
||||
@ -550,7 +553,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
|
||||
setStarted(false);
|
||||
|
||||
LargeMessageDeliverer del = largeMessageDeliverer;
|
||||
CoreLargeMessageDeliverer del = largeMessageDeliverer;
|
||||
|
||||
if (del != null) {
|
||||
del.finish();
|
||||
@ -688,7 +691,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
boolean performACK = lastConsumedAsDelivered;
|
||||
|
||||
try {
|
||||
LargeMessageDeliverer pendingLargeMessageDeliverer = largeMessageDeliverer;
|
||||
CoreLargeMessageDeliverer pendingLargeMessageDeliverer = largeMessageDeliverer;
|
||||
if (pendingLargeMessageDeliverer != null) {
|
||||
pendingLargeMessageDeliverer.finish();
|
||||
}
|
||||
@ -1220,7 +1223,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
* Internal encapsulation of the logic on sending LargeMessages.
|
||||
* This Inner class was created to avoid a bunch of loose properties about the current LargeMessage being sent
|
||||
*/
|
||||
private final class LargeMessageDeliverer {
|
||||
private final class CoreLargeMessageDeliverer {
|
||||
|
||||
private long sizePendingLargeMessage;
|
||||
|
||||
@ -1235,14 +1238,14 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
*/
|
||||
private long positionPendingLargeMessage;
|
||||
|
||||
private LargeBodyEncoder context;
|
||||
private LargeBodyReader context;
|
||||
|
||||
private ByteBuffer chunkBytes;
|
||||
|
||||
private LargeMessageDeliverer(final LargeServerMessage message, final MessageReference ref) throws Exception {
|
||||
private CoreLargeMessageDeliverer(final LargeServerMessage message, final MessageReference ref) throws Exception {
|
||||
largeMessage = message;
|
||||
|
||||
largeMessage.incrementDelayDeletionCount();
|
||||
largeMessage.toMessage().usageUp();
|
||||
|
||||
this.ref = ref;
|
||||
|
||||
@ -1289,15 +1292,15 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
}
|
||||
|
||||
if (!sentInitialPacket) {
|
||||
context = currentLargeMessage.getBodyEncoder();
|
||||
context = currentLargeMessage.getLargeBodyReader();
|
||||
|
||||
sizePendingLargeMessage = context.getLargeBodySize();
|
||||
sizePendingLargeMessage = context.getSize();
|
||||
|
||||
context.open();
|
||||
|
||||
sentInitialPacket = true;
|
||||
|
||||
int packetSize = callback.sendLargeMessage(ref, currentLargeMessage, ServerConsumerImpl.this, context.getLargeBodySize(), ref.getDeliveryCount());
|
||||
int packetSize = callback.sendLargeMessage(ref, currentLargeMessage.toMessage(), ServerConsumerImpl.this, context.getSize(), ref.getDeliveryCount());
|
||||
|
||||
if (availableCredits != null) {
|
||||
final int credits = availableCredits.addAndGet(-packetSize);
|
||||
@ -1337,7 +1340,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
|
||||
assert bodyBuffer.remaining() == localChunkLen;
|
||||
|
||||
final int readBytes = context.encode(bodyBuffer);
|
||||
final int readBytes = context.readInto(bodyBuffer);
|
||||
|
||||
assert readBytes == localChunkLen;
|
||||
|
||||
@ -1406,12 +1409,7 @@ public class ServerConsumerImpl implements ServerConsumer, ReadyListener {
|
||||
|
||||
largeMessage.releaseResources(false);
|
||||
|
||||
largeMessage.decrementDelayDeletionCount();
|
||||
|
||||
if (preAcknowledge && !browseOnly) {
|
||||
// PreAck will have an extra reference
|
||||
largeMessage.decrementDelayDeletionCount();
|
||||
}
|
||||
largeMessage.toMessage().usageDown();
|
||||
|
||||
largeMessageDeliverer = null;
|
||||
|
||||
|
@ -73,22 +73,22 @@ public class ServerMessageImpl extends MessageInternalImpl implements ServerMess
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementRefCount() throws Exception {
|
||||
public int refUp() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementRefCount() throws Exception {
|
||||
public int refDown() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementDurableRefCount() {
|
||||
public int durableUp() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementDurableRefCount() {
|
||||
public int durableDown() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
|
@ -18,14 +18,23 @@
|
||||
package org.apache.activemq.artemis.spi.core.protocol;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.RefCountMessage;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.journal.AbstractJournalStorageManager;
|
||||
import org.apache.activemq.artemis.core.persistence.impl.journal.LargeServerMessageImpl;
|
||||
import org.apache.activemq.artemis.core.server.LargeServerMessage;
|
||||
import org.jboss.logging.Logger;
|
||||
|
||||
public class EmbedMessageUtil {
|
||||
|
||||
|
||||
private static final String AMQP_ENCODE_PROPERTY = "_AMQP_EMBED_LARGE";
|
||||
|
||||
private static final byte[] signature = new byte[]{(byte) 'E', (byte) 'M', (byte) 'B'};
|
||||
|
||||
private static final Logger logger = Logger.getLogger(EmbedMessageUtil.class);
|
||||
@ -35,35 +44,76 @@ public class EmbedMessageUtil {
|
||||
if (source instanceof ICoreMessage) {
|
||||
return (ICoreMessage) source;
|
||||
} else {
|
||||
Persister persister = source.getPersister();
|
||||
|
||||
CoreMessage message = new CoreMessage(source.getMessageID(), persister.getEncodeSize(source) + signature.length + CoreMessage.BODY_OFFSET).setType(Message.EMBEDDED_TYPE);
|
||||
if (source.isLargeMessage()) {
|
||||
LargeServerMessage largeSource = (LargeServerMessage) source;
|
||||
|
||||
ActiveMQBuffer buffer = message.getBodyBuffer();
|
||||
buffer.writeBytes(signature);
|
||||
persister.encode(buffer, source);
|
||||
return message;
|
||||
LargeServerMessageImpl largeServerMessage = new LargeServerMessageImpl(Message.LARGE_EMBEDDED_TYPE, source.getMessageID(), largeSource.getStorageManager(), largeSource.getLargeBody().createFile());
|
||||
largeServerMessage.setDurable(source.isDurable());
|
||||
int size = source.getPersister().getEncodeSize(source);
|
||||
byte[] arrayByte = new byte[size];
|
||||
ActiveMQBuffer buffer = ActiveMQBuffers.wrappedBuffer(arrayByte);
|
||||
buffer.resetWriterIndex();
|
||||
source.getPersister().encode(buffer, source);
|
||||
largeServerMessage.toMessage().putBytesProperty(AMQP_ENCODE_PROPERTY, arrayByte);
|
||||
largeServerMessage.setParentRef((RefCountMessage)source);
|
||||
return (ICoreMessage) largeServerMessage.toMessage();
|
||||
} else {
|
||||
Persister persister = source.getPersister();
|
||||
|
||||
CoreMessage message = new CoreMessage(source.getMessageID(), persister.getEncodeSize(source) + signature.length + CoreMessage.BODY_OFFSET).setType(Message.EMBEDDED_TYPE);
|
||||
message.setDurable(source.isDurable());
|
||||
|
||||
ActiveMQBuffer buffer = message.getBodyBuffer();
|
||||
buffer.writeBytes(signature);
|
||||
persister.encode(buffer, source);
|
||||
return message;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static Message extractEmbedded(ICoreMessage message) {
|
||||
if (message.getType() == Message.EMBEDDED_TYPE) {
|
||||
ActiveMQBuffer buffer = message.getReadOnlyBodyBuffer();
|
||||
|
||||
if (buffer.readableBytes() < signature.length || !checkSignature(buffer)) {
|
||||
logger.tracef("Message type %d was used for something other than embed messages, ignoring content and treating as a regular message", Message.EMBEDDED_TYPE);
|
||||
public static Message extractEmbedded(ICoreMessage message, StorageManager storageManager) {
|
||||
switch (message.getType()) {
|
||||
case Message.EMBEDDED_TYPE:
|
||||
return extractRegularMessage(message, storageManager);
|
||||
case Message.LARGE_EMBEDDED_TYPE:
|
||||
return extractLargeMessage(message, storageManager);
|
||||
default:
|
||||
return message;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
return MessagePersister.getInstance().decode(buffer, null);
|
||||
} catch (Exception e) {
|
||||
logger.warn(e.getMessage(), e);
|
||||
return message;
|
||||
}
|
||||
} else {
|
||||
private static Message extractRegularMessage(ICoreMessage message, StorageManager storageManager) {
|
||||
ActiveMQBuffer buffer = message.getReadOnlyBodyBuffer();
|
||||
|
||||
if (buffer.readableBytes() < signature.length || !checkSignature(buffer)) {
|
||||
logger.tracef("Message type %d was used for something other than embed messages, ignoring content and treating as a regular message", Message.EMBEDDED_TYPE);
|
||||
return message;
|
||||
}
|
||||
|
||||
return readEncoded(message, storageManager, buffer);
|
||||
}
|
||||
|
||||
private static Message readEncoded(ICoreMessage message, StorageManager storageManager, ActiveMQBuffer buffer) {
|
||||
|
||||
|
||||
AbstractJournalStorageManager.setupThreadLocal(storageManager);
|
||||
try {
|
||||
Message returnMessage = MessagePersister.getInstance().decode(buffer, null, null);
|
||||
returnMessage.setMessageID(message.getMessageID());
|
||||
return returnMessage;
|
||||
} catch (Exception e) {
|
||||
logger.warn(e.getMessage(), e);
|
||||
return message;
|
||||
} finally {
|
||||
AbstractJournalStorageManager.setupThreadLocal(null);
|
||||
}
|
||||
}
|
||||
|
||||
private static Message extractLargeMessage(ICoreMessage message, StorageManager storageManager) {
|
||||
ActiveMQBuffer buffer = ActiveMQBuffers.wrappedBuffer(message.getBytesProperty(AMQP_ENCODE_PROPERTY));
|
||||
|
||||
return readEncoded(message, storageManager, buffer);
|
||||
}
|
||||
|
||||
private static boolean checkSignature(final ActiveMQBuffer buffer) {
|
||||
|
@ -18,11 +18,12 @@ package org.apache.activemq.artemis.spi.core.protocol;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.StorageManager;
|
||||
|
||||
public interface MessageConverter<ProtocolMessage extends Message> {
|
||||
|
||||
ICoreMessage toCore(ProtocolMessage pureMessage, CoreMessageObjectPools coreMessageObjectPools) throws Exception;
|
||||
|
||||
ProtocolMessage fromCore(ICoreMessage coreMessage) throws Exception;
|
||||
ProtocolMessage fromCore(ICoreMessage coreMessage, StorageManager storageManager) throws Exception;
|
||||
}
|
||||
|
@ -21,20 +21,26 @@ import java.util.ServiceLoader;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessagePersister;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.jboss.logging.Logger;
|
||||
|
||||
public class MessagePersister implements Persister<Message, CoreMessageObjectPools> {
|
||||
import static org.apache.activemq.artemis.core.persistence.PersisterIDs.MAX_PERSISTERS;
|
||||
|
||||
public class MessagePersister implements Persister<Message> {
|
||||
|
||||
@Override
|
||||
public byte getID() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
private static final Logger logger = Logger.getLogger(MessagePersister.class);
|
||||
|
||||
private static final MessagePersister theInstance = new MessagePersister();
|
||||
|
||||
/** This will be used for reading messages */
|
||||
private static final int MAX_PERSISTERS = 3;
|
||||
private static final Persister<Message, CoreMessageObjectPools>[] persisters = new Persister[MAX_PERSISTERS];
|
||||
private static final Persister<Message>[] persisters = new Persister[MAX_PERSISTERS];
|
||||
|
||||
static {
|
||||
CoreMessagePersister persister = CoreMessagePersister.getInstance();
|
||||
@ -47,7 +53,7 @@ public class MessagePersister implements Persister<Message, CoreMessageObjectPoo
|
||||
}
|
||||
|
||||
public static void registerProtocol(ProtocolManagerFactory manager) {
|
||||
Persister<Message, CoreMessageObjectPools>[] messagePersisters = manager.getPersister();
|
||||
Persister<Message>[] messagePersisters = manager.getPersister();
|
||||
if (messagePersisters == null || messagePersisters.length == 0) {
|
||||
logger.debug("Cannot find persister for " + manager);
|
||||
} else {
|
||||
@ -70,7 +76,7 @@ public class MessagePersister implements Persister<Message, CoreMessageObjectPoo
|
||||
return persisters[id - 1];
|
||||
}
|
||||
|
||||
public static void registerPersister(Persister<Message, CoreMessageObjectPools> persister) {
|
||||
public static void registerPersister(Persister<Message> persister) {
|
||||
if (persister != null) {
|
||||
assert persister.getID() <= MAX_PERSISTERS : "You must update MessagePersister::MAX_PERSISTERS to a higher number";
|
||||
persisters[persister.getID() - 1] = persister;
|
||||
@ -98,12 +104,12 @@ public class MessagePersister implements Persister<Message, CoreMessageObjectPoo
|
||||
}
|
||||
|
||||
@Override
|
||||
public Message decode(ActiveMQBuffer buffer, CoreMessageObjectPools pools) {
|
||||
public Message decode(ActiveMQBuffer buffer, Message record, CoreMessageObjectPools pools) {
|
||||
byte protocol = buffer.readByte();
|
||||
Persister<Message, CoreMessageObjectPools> persister = getPersister(protocol);
|
||||
Persister<Message> persister = getPersister(protocol);
|
||||
if (persister == null) {
|
||||
throw new NullPointerException("couldn't find factory for type=" + protocol);
|
||||
}
|
||||
return persister.decode(buffer, pools);
|
||||
return persister.decode(buffer, record, pools);
|
||||
}
|
||||
}
|
||||
|
@ -21,13 +21,12 @@ import java.util.Map;
|
||||
|
||||
import org.apache.activemq.artemis.api.core.BaseInterceptor;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||
|
||||
public interface ProtocolManagerFactory<P extends BaseInterceptor> {
|
||||
|
||||
default Persister<Message, CoreMessageObjectPools>[] getPersister() {
|
||||
default Persister<Message>[] getPersister() {
|
||||
return new Persister[]{};
|
||||
}
|
||||
|
||||
|
@ -76,7 +76,7 @@ public class QueueImplTest {
|
||||
return null;
|
||||
}).when(storageManager).afterCompleteOperations(Mockito.any(IOCallback.class));
|
||||
|
||||
QueueImpl queue = new QueueImpl(0, address, address, null, pageSubscription, null, false,
|
||||
QueueImpl queue = new QueueImpl(0, address, address, null, null, pageSubscription, null, false,
|
||||
false, false, Mockito.mock(ScheduledExecutorService.class),
|
||||
Mockito.mock(PostOffice.class), storageManager, null,
|
||||
Mockito.mock(ArtemisExecutor.class), Mockito.mock(ActiveMQServer.class),
|
||||
|
@ -36,12 +36,12 @@ import org.apache.activemq.artemis.api.core.ActiveMQException;
|
||||
import org.apache.activemq.artemis.api.core.ActiveMQPropertyConversionException;
|
||||
import org.apache.activemq.artemis.api.core.Message;
|
||||
import org.apache.activemq.artemis.api.core.Pair;
|
||||
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.filter.Filter;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessage;
|
||||
import org.apache.activemq.artemis.core.message.impl.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.persistence.CoreMessageObjectPools;
|
||||
import org.apache.activemq.artemis.core.paging.PagingStore;
|
||||
import org.apache.activemq.artemis.core.paging.cursor.PageSubscription;
|
||||
import org.apache.activemq.artemis.core.persistence.OperationContext;
|
||||
import org.apache.activemq.artemis.core.persistence.Persister;
|
||||
@ -290,7 +290,7 @@ public class ScheduledDeliveryHandlerTest extends Assert {
|
||||
}
|
||||
}
|
||||
|
||||
class FakeMessage extends RefCountMessage {
|
||||
class FakeMessage implements Message {
|
||||
|
||||
@Override
|
||||
public SimpleString getReplyTo() {
|
||||
@ -317,6 +317,11 @@ public class ScheduledDeliveryHandlerTest extends Assert {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getDurableCount() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long getScheduledDeliveryTime() {
|
||||
return null;
|
||||
@ -328,7 +333,7 @@ public class ScheduledDeliveryHandlerTest extends Assert {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Persister<Message, CoreMessageObjectPools> getPersister() {
|
||||
public Persister<Message> getPersister() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@ -363,22 +368,12 @@ public class ScheduledDeliveryHandlerTest extends Assert {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementRefCount() throws Exception {
|
||||
public int durableUp() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementRefCount() throws Exception {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int incrementDurableRefCount() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int decrementDurableRefCount() {
|
||||
public int durableDown() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@ -776,6 +771,31 @@ public class ScheduledDeliveryHandlerTest extends Assert {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUsage() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int usageUp() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int usageDown() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int refUp() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int refDown() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sendBuffer(ByteBuf buffer, int count) {
|
||||
|
||||
@ -794,6 +814,31 @@ public class ScheduledDeliveryHandlerTest extends Assert {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public PagingStore getPagingStore() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int durableUp(Message message) {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int durableDown(Message message) {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void refUp(Message message) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void refDown(Message message) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeAddress() throws Exception {
|
||||
}
|
||||
|
@ -26,6 +26,7 @@ import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
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.Pair;
|
||||
@ -240,6 +241,16 @@ public class TransactionImplTest extends ActiveMQTestBase {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteLargeMessageBody(LargeServerMessage largeServerMessage) throws ActiveMQException {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addBytesToLargeMessage(SequentialFile file, long messageId, ActiveMQBuffer bytes) throws Exception {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pageClosed(SimpleString storeName, int pageNumber) {
|
||||
|
||||
|
@ -1942,26 +1942,16 @@ public abstract class ActiveMQTestBase extends Assert {
|
||||
return false;
|
||||
}
|
||||
|
||||
protected int getNumberOfFiles(File directory) {
|
||||
return directory.listFiles().length;
|
||||
}
|
||||
/**
|
||||
* Deleting a file on LargeDir is an asynchronous process. We need to keep looking for a while if
|
||||
* the file hasn't been deleted yet.
|
||||
*/
|
||||
protected void validateNoFilesOnLargeDir(final String directory, final int expect) throws Exception {
|
||||
File largeMessagesFileDir = new File(directory);
|
||||
|
||||
// Deleting the file is async... we keep looking for a period of the time until the file is really gone
|
||||
long timeout = System.currentTimeMillis() + 5000;
|
||||
while (timeout > System.currentTimeMillis() && largeMessagesFileDir.listFiles().length != expect) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
if (expect != largeMessagesFileDir.listFiles().length) {
|
||||
for (File file : largeMessagesFileDir.listFiles()) {
|
||||
System.out.println("File " + file + " still on ");
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertEquals(expect, largeMessagesFileDir.listFiles().length);
|
||||
Wait.assertEquals(expect, () -> getNumberOfFiles(largeMessagesFileDir));
|
||||
}
|
||||
|
||||
/**
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user