ARTEMIS-1780 Handle conversion of large ObjectMessage types
Make sure the correct buffer is used when decoding the stored Core message that originated from the conversion of an AMQP message sent and annotated as a JMS ObjectMessage which trips the large message boundary.
This commit is contained in:
parent
32090121d2
commit
fc32bc0b07
|
@ -17,14 +17,31 @@
|
||||||
|
|
||||||
package org.apache.activemq.artemis.protocol.amqp.converter;
|
package org.apache.activemq.artemis.protocol.amqp.converter;
|
||||||
|
|
||||||
import javax.jms.Destination;
|
import static org.apache.activemq.artemis.api.core.FilterConstants.NATIVE_MESSAGE_ID;
|
||||||
import javax.jms.JMSException;
|
import static org.apache.activemq.artemis.api.core.Message.HDR_SCHEDULED_DELIVERY_TIME;
|
||||||
import javax.jms.MessageEOFException;
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.EMPTY_BINARY;
|
||||||
import javax.jms.Queue;
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_CONTENT_ENCODING;
|
||||||
import javax.jms.TemporaryQueue;
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_CONTENT_TYPE;
|
||||||
import javax.jms.TemporaryTopic;
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_DELIVERY_ANNOTATION_PREFIX;
|
||||||
import javax.jms.TextMessage;
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_FIRST_ACQUIRER;
|
||||||
import javax.jms.Topic;
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_FOOTER_PREFIX;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER_DURABLE;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER_PRIORITY;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_MESSAGE_ANNOTATION_PREFIX;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_NATIVE;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_PREFIX;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_PROPERTIES;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_REPLYTO_GROUP_ID;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_DEST_TYPE_MSG_ANNOTATION;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_REPLY_TO_TYPE_MSG_ANNOTATION;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.QUEUE_TYPE;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TEMP_QUEUE_TYPE;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TEMP_TOPIC_TYPE;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TOPIC_TYPE;
|
||||||
|
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.toAddress;
|
||||||
|
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
|
@ -34,8 +51,15 @@ import java.util.LinkedHashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
import io.netty.buffer.ByteBuf;
|
import javax.jms.Destination;
|
||||||
import io.netty.buffer.PooledByteBufAllocator;
|
import javax.jms.JMSException;
|
||||||
|
import javax.jms.MessageEOFException;
|
||||||
|
import javax.jms.Queue;
|
||||||
|
import javax.jms.TemporaryQueue;
|
||||||
|
import javax.jms.TemporaryTopic;
|
||||||
|
import javax.jms.TextMessage;
|
||||||
|
import javax.jms.Topic;
|
||||||
|
|
||||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||||
import org.apache.activemq.artemis.api.core.Message;
|
import org.apache.activemq.artemis.api.core.Message;
|
||||||
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
import org.apache.activemq.artemis.protocol.amqp.broker.AMQPMessage;
|
||||||
|
@ -67,30 +91,8 @@ import org.apache.qpid.proton.codec.EncoderImpl;
|
||||||
import org.apache.qpid.proton.codec.WritableBuffer;
|
import org.apache.qpid.proton.codec.WritableBuffer;
|
||||||
import org.jboss.logging.Logger;
|
import org.jboss.logging.Logger;
|
||||||
|
|
||||||
import static org.apache.activemq.artemis.api.core.FilterConstants.NATIVE_MESSAGE_ID;
|
import io.netty.buffer.ByteBuf;
|
||||||
import static org.apache.activemq.artemis.api.core.Message.HDR_SCHEDULED_DELIVERY_TIME;
|
import io.netty.buffer.PooledByteBufAllocator;
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.EMPTY_BINARY;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_CONTENT_ENCODING;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_CONTENT_TYPE;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_DELIVERY_ANNOTATION_PREFIX;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_FIRST_ACQUIRER;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_FOOTER_PREFIX;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER_DURABLE;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_HEADER_PRIORITY;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_MESSAGE_ANNOTATION_PREFIX;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_NATIVE;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_PREFIX;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_PROPERTIES;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_AMQP_REPLYTO_GROUP_ID;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_DEST_TYPE_MSG_ANNOTATION;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.JMS_REPLY_TO_TYPE_MSG_ANNOTATION;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.QUEUE_TYPE;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.SERIALIZED_JAVA_OBJECT_CONTENT_TYPE;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TEMP_QUEUE_TYPE;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TEMP_TOPIC_TYPE;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.TOPIC_TYPE;
|
|
||||||
import static org.apache.activemq.artemis.protocol.amqp.converter.AMQPMessageSupport.toAddress;
|
|
||||||
|
|
||||||
public class CoreAmqpConverter {
|
public class CoreAmqpConverter {
|
||||||
|
|
||||||
|
@ -405,12 +407,7 @@ public class CoreAmqpConverter {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Binary getBinaryFromMessageBody(ServerJMSObjectMessage message) throws JMSException {
|
private static Binary getBinaryFromMessageBody(ServerJMSObjectMessage message) throws JMSException {
|
||||||
message.getInnerMessage().getBodyBuffer().resetReaderIndex();
|
return message.getSerializedForm();
|
||||||
int size = message.getInnerMessage().getBodyBuffer().readInt();
|
|
||||||
byte[] bytes = new byte[size];
|
|
||||||
message.getInnerMessage().getBodyBuffer().readBytes(bytes);
|
|
||||||
|
|
||||||
return new Binary(bytes);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Map<String, Object> getMapFromMessageBody(ServerJMSMapMessage message) throws JMSException {
|
private static Map<String, Object> getMapFromMessageBody(ServerJMSMapMessage message) throws JMSException {
|
||||||
|
|
|
@ -16,10 +16,12 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.activemq.artemis.protocol.amqp.converter.jms;
|
package org.apache.activemq.artemis.protocol.amqp.converter.jms;
|
||||||
|
|
||||||
import javax.jms.JMSException;
|
|
||||||
import javax.jms.ObjectMessage;
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
|
||||||
|
import javax.jms.JMSException;
|
||||||
|
import javax.jms.ObjectMessage;
|
||||||
|
|
||||||
|
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
|
||||||
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
import org.apache.activemq.artemis.api.core.ICoreMessage;
|
||||||
import org.apache.activemq.artemis.api.core.Message;
|
import org.apache.activemq.artemis.api.core.Message;
|
||||||
import org.apache.qpid.proton.amqp.Binary;
|
import org.apache.qpid.proton.amqp.Binary;
|
||||||
|
@ -62,9 +64,10 @@ public class ServerJMSObjectMessage extends ServerJMSMessage implements ObjectMe
|
||||||
@Override
|
@Override
|
||||||
public void decode() throws Exception {
|
public void decode() throws Exception {
|
||||||
super.decode();
|
super.decode();
|
||||||
int size = getInnerMessage().getBodyBuffer().readInt();
|
ActiveMQBuffer buffer = getInnerMessage().getDataBuffer();
|
||||||
|
int size = buffer.readInt();
|
||||||
byte[] bytes = new byte[size];
|
byte[] bytes = new byte[size];
|
||||||
getInnerMessage().getBodyBuffer().readBytes(bytes);
|
buffer.readBytes(bytes);
|
||||||
payload = new Binary(bytes);
|
payload = new Binary(bytes);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,14 +16,21 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.activemq.artemis.tests.integration.amqp;
|
package org.apache.activemq.artemis.tests.integration.amqp;
|
||||||
|
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
import javax.jms.BytesMessage;
|
||||||
import javax.jms.Connection;
|
import javax.jms.Connection;
|
||||||
import javax.jms.ConnectionFactory;
|
import javax.jms.ConnectionFactory;
|
||||||
import javax.jms.JMSException;
|
import javax.jms.JMSException;
|
||||||
import javax.jms.Message;
|
import javax.jms.Message;
|
||||||
import javax.jms.MessageConsumer;
|
import javax.jms.MessageConsumer;
|
||||||
|
import javax.jms.MessageProducer;
|
||||||
|
import javax.jms.ObjectMessage;
|
||||||
|
import javax.jms.Queue;
|
||||||
import javax.jms.Session;
|
import javax.jms.Session;
|
||||||
import java.util.Map;
|
import javax.jms.TextMessage;
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
import org.apache.activemq.artemis.api.core.RoutingType;
|
import org.apache.activemq.artemis.api.core.RoutingType;
|
||||||
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
import org.apache.activemq.artemis.core.server.ActiveMQServer;
|
||||||
|
@ -35,6 +42,7 @@ import org.apache.activemq.transport.amqp.client.AmqpMessage;
|
||||||
import org.apache.activemq.transport.amqp.client.AmqpReceiver;
|
import org.apache.activemq.transport.amqp.client.AmqpReceiver;
|
||||||
import org.apache.activemq.transport.amqp.client.AmqpSender;
|
import org.apache.activemq.transport.amqp.client.AmqpSender;
|
||||||
import org.apache.activemq.transport.amqp.client.AmqpSession;
|
import org.apache.activemq.transport.amqp.client.AmqpSession;
|
||||||
|
import org.apache.qpid.jms.JmsConnectionFactory;
|
||||||
import org.apache.qpid.proton.amqp.messaging.Data;
|
import org.apache.qpid.proton.amqp.messaging.Data;
|
||||||
import org.apache.qpid.proton.message.impl.MessageImpl;
|
import org.apache.qpid.proton.message.impl.MessageImpl;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -56,14 +64,11 @@ public class AmqpLargeMessageTest extends AmqpClientTestSupport {
|
||||||
protected void createAddressAndQueues(ActiveMQServer server) throws Exception {
|
protected void createAddressAndQueues(ActiveMQServer server) throws Exception {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void addAdditionalAcceptors(ActiveMQServer server) throws Exception {
|
protected void addAdditionalAcceptors(ActiveMQServer server) throws Exception {
|
||||||
//server.getConfiguration().addAcceptorConfiguration("tcp", "tcp://localhost:5445");
|
|
||||||
server.getConfiguration().addAcceptorConfiguration("tcp", "tcp://localhost:61616");
|
server.getConfiguration().addAcceptorConfiguration("tcp", "tcp://localhost:61616");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
public void testSendAMQPReceiveCore() throws Exception {
|
public void testSendAMQPReceiveCore() throws Exception {
|
||||||
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
|
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
|
||||||
|
@ -85,7 +90,6 @@ public class AmqpLargeMessageTest extends AmqpClientTestSupport {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test(timeout = 60000)
|
@Test(timeout = 60000)
|
||||||
public void testSendAMQPReceiveOpenWire() throws Exception {
|
public void testSendAMQPReceiveOpenWire() throws Exception {
|
||||||
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
|
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
|
||||||
|
@ -177,6 +181,120 @@ public class AmqpLargeMessageTest extends AmqpClientTestSupport {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60000)
|
||||||
|
public void testSendAMQPReceiveAMQPViaJMSObjectMessage() throws Exception {
|
||||||
|
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
|
||||||
|
|
||||||
|
String testQueueName = "ConnectionFrameSize";
|
||||||
|
int nMsgs = 1;
|
||||||
|
|
||||||
|
ConnectionFactory factory = new JmsConnectionFactory("amqp://localhost:61616");
|
||||||
|
|
||||||
|
sendObjectMessages(nMsgs, new JmsConnectionFactory("amqp://localhost:61616"));
|
||||||
|
|
||||||
|
int count = getMessageCount(server.getPostOffice(), testQueueName);
|
||||||
|
assertEquals(nMsgs, count);
|
||||||
|
|
||||||
|
receiveJMS(nMsgs, factory);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60000)
|
||||||
|
public void testSendAMQPReceiveAMQPViaJMSText() throws Exception {
|
||||||
|
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
|
||||||
|
|
||||||
|
String testQueueName = "ConnectionFrameSize";
|
||||||
|
int nMsgs = 1;
|
||||||
|
|
||||||
|
ConnectionFactory factory = new JmsConnectionFactory("amqp://localhost:61616");
|
||||||
|
|
||||||
|
sendTextMessages(nMsgs, new JmsConnectionFactory("amqp://localhost:61616"));
|
||||||
|
|
||||||
|
int count = getMessageCount(server.getPostOffice(), testQueueName);
|
||||||
|
assertEquals(nMsgs, count);
|
||||||
|
|
||||||
|
receiveJMS(nMsgs, factory);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60000)
|
||||||
|
public void testSendAMQPReceiveAMQPViaJMSBytes() throws Exception {
|
||||||
|
server.getAddressSettingsRepository().addMatch("#", new AddressSettings().setDefaultAddressRoutingType(RoutingType.ANYCAST));
|
||||||
|
|
||||||
|
String testQueueName = "ConnectionFrameSize";
|
||||||
|
int nMsgs = 1;
|
||||||
|
|
||||||
|
ConnectionFactory factory = new JmsConnectionFactory("amqp://localhost:61616");
|
||||||
|
|
||||||
|
sendBytesMessages(nMsgs, new JmsConnectionFactory("amqp://localhost:61616"));
|
||||||
|
|
||||||
|
int count = getMessageCount(server.getPostOffice(), testQueueName);
|
||||||
|
assertEquals(nMsgs, count);
|
||||||
|
|
||||||
|
receiveJMS(nMsgs, factory);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void sendObjectMessages(int nMsgs, ConnectionFactory factory) throws Exception {
|
||||||
|
try (Connection connection = factory.createConnection()) {
|
||||||
|
Session session = connection.createSession();
|
||||||
|
Queue queue = session.createQueue(testQueueName);
|
||||||
|
MessageProducer producer = session.createProducer(queue);
|
||||||
|
ObjectMessage msg = session.createObjectMessage();
|
||||||
|
|
||||||
|
StringBuilder builder = new StringBuilder();
|
||||||
|
for (int i = 0; i < PAYLOAD; ++i) {
|
||||||
|
builder.append("A");
|
||||||
|
}
|
||||||
|
|
||||||
|
msg.setObject(builder.toString());
|
||||||
|
|
||||||
|
for (int i = 0; i < nMsgs; ++i) {
|
||||||
|
msg.setIntProperty("i", (Integer) i);
|
||||||
|
producer.send(msg);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void sendTextMessages(int nMsgs, ConnectionFactory factory) throws Exception {
|
||||||
|
try (Connection connection = factory.createConnection()) {
|
||||||
|
Session session = connection.createSession();
|
||||||
|
Queue queue = session.createQueue(testQueueName);
|
||||||
|
MessageProducer producer = session.createProducer(queue);
|
||||||
|
TextMessage msg = session.createTextMessage();
|
||||||
|
|
||||||
|
StringBuilder builder = new StringBuilder();
|
||||||
|
for (int i = 0; i < PAYLOAD; ++i) {
|
||||||
|
builder.append("A");
|
||||||
|
}
|
||||||
|
|
||||||
|
msg.setText(builder.toString());
|
||||||
|
|
||||||
|
for (int i = 0; i < nMsgs; ++i) {
|
||||||
|
msg.setIntProperty("i", (Integer) i);
|
||||||
|
producer.send(msg);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void sendBytesMessages(int nMsgs, ConnectionFactory factory) throws Exception {
|
||||||
|
try (Connection connection = factory.createConnection()) {
|
||||||
|
Session session = connection.createSession();
|
||||||
|
Queue queue = session.createQueue(testQueueName);
|
||||||
|
MessageProducer producer = session.createProducer(queue);
|
||||||
|
BytesMessage msg = session.createBytesMessage();
|
||||||
|
|
||||||
|
StringBuilder builder = new StringBuilder();
|
||||||
|
for (int i = 0; i < PAYLOAD; ++i) {
|
||||||
|
builder.append("A");
|
||||||
|
}
|
||||||
|
|
||||||
|
msg.writeBytes(builder.toString().getBytes(StandardCharsets.UTF_8));
|
||||||
|
|
||||||
|
for (int i = 0; i < nMsgs; ++i) {
|
||||||
|
msg.setIntProperty("i", (Integer) i);
|
||||||
|
producer.send(msg);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private AmqpMessage createAmqpMessage(byte value, int payloadSize) {
|
private AmqpMessage createAmqpMessage(byte value, int payloadSize) {
|
||||||
AmqpMessage message = new AmqpMessage();
|
AmqpMessage message = new AmqpMessage();
|
||||||
byte[] payload = new byte[payloadSize];
|
byte[] payload = new byte[payloadSize];
|
||||||
|
|
Loading…
Reference in New Issue