This commit is contained in:
Clebert Suconic 2017-10-09 12:51:02 -04:00
commit 16aa7ed6cf
15 changed files with 174 additions and 11 deletions

View File

@ -120,6 +120,11 @@ public class JDBCSequentialFileFactory implements SequentialFileFactory, ActiveM
return false;
}
@Override
public long getBufferSize() {
return dbDriver.getMaxSize();
}
@Override
public synchronized void start() {
try {

View File

@ -127,6 +127,16 @@ public class JDBCJournalImpl extends AbstractJDBCDriver implements Journal {
public void flush() throws Exception {
}
/**
* The max size record that can be stored in the journal
*
* @return
*/
@Override
public long getMaxRecordSize() {
return sqlProvider.getMaxBlobSize();
}
@Override
protected void createSchema() throws SQLException {
createTable(sqlProvider.getCreateJournalTableSQL());

View File

@ -101,4 +101,6 @@ public interface SequentialFileFactory {
SequentialFileFactory setDatasync(boolean enabled);
boolean isDatasync();
long getBufferSize();
}

View File

@ -269,6 +269,11 @@ public final class AIOSequentialFileFactory extends AbstractSequentialFileFactor
}
}
@Override
public long getBufferSize() {
return bufferSize;
}
/**
* The same callback is used for Runnable executor.
* This way we can save some memory over the pool.

View File

@ -41,6 +41,7 @@ public final class MappedSequentialFileFactory implements SequentialFileFactory
private boolean bufferPooling;
//pools only the biggest one -> optimized for the common case
private final ThreadLocal<ByteBuffer> bytesPool;
private final int bufferSize;
private MappedSequentialFileFactory(File directory,
int capacity,
@ -57,6 +58,7 @@ public final class MappedSequentialFileFactory implements SequentialFileFactory
} else {
timedBuffer = null;
}
this.bufferSize = bufferSize;
this.bufferPooling = true;
this.bytesPool = new ThreadLocal<>();
}
@ -105,6 +107,11 @@ public final class MappedSequentialFileFactory implements SequentialFileFactory
return useDataSync;
}
@Override
public long getBufferSize() {
return bufferSize;
}
@Override
public int getMaxIO() {
return 1;

View File

@ -203,4 +203,9 @@ public final class NIOSequentialFileFactory extends AbstractSequentialFileFactor
return bytes;
}
@Override
public long getBufferSize() {
return bufferSize;
}
}

View File

@ -279,4 +279,10 @@ public interface Journal extends ActiveMQComponent {
* It will make sure there are no more pending operations on the Executors.
* */
void flush() throws Exception;
/**
* The max size record that can be stored in the journal
* @return
*/
long getMaxRecordSize();
}

View File

@ -103,6 +103,16 @@ public final class FileWrapperJournal extends JournalBase {
public void flush() throws Exception {
}
/**
* The max size record that can be stored in the journal
*
* @return
*/
@Override
public long getMaxRecordSize() {
return journal.getMaxRecordSize();
}
/**
* Write the record to the current file.
*/

View File

@ -2200,6 +2200,16 @@ public class JournalImpl extends JournalBase implements TestableJournal, Journal
flushExecutor(compactorExecutor);
}
/**
* The max size record that can be stored in the journal
*
* @return
*/
@Override
public long getMaxRecordSize() {
return Math.min(getFileSize(), fileFactory.getBufferSize());
}
private void flushExecutor(Executor executor) throws InterruptedException {
if (executor != null) {

View File

@ -548,4 +548,9 @@ public class ReplicatedJournal implements Journal {
public void replicationSyncFinished() {
throw new UnsupportedOperationException("should never get called");
}
@Override
public long getMaxRecordSize() {
return localJournal.getMaxRecordSize();
}
}

View File

@ -16,13 +16,12 @@
*/
package org.apache.activemq.artemis.core.server;
import javax.json.JsonArrayBuilder;
import javax.transaction.xa.Xid;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.json.JsonArrayBuilder;
import javax.transaction.xa.Xid;
import org.apache.activemq.artemis.Closeable;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.Pair;

View File

@ -16,8 +16,10 @@
*/
package org.apache.activemq.artemis.core.server.impl;
import static org.apache.activemq.artemis.api.core.JsonUtil.nullSafe;
import javax.json.JsonArrayBuilder;
import javax.json.JsonObjectBuilder;
import javax.transaction.xa.XAException;
import javax.transaction.xa.Xid;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@ -29,16 +31,13 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import javax.json.JsonArrayBuilder;
import javax.json.JsonObjectBuilder;
import javax.transaction.xa.XAException;
import javax.transaction.xa.Xid;
import org.apache.activemq.artemis.Closeable;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQException;
import org.apache.activemq.artemis.api.core.ActiveMQIOErrorException;
import org.apache.activemq.artemis.api.core.ActiveMQIllegalStateException;
import org.apache.activemq.artemis.api.core.ActiveMQNonExistentQueueException;
import org.apache.activemq.artemis.api.core.ICoreMessage;
import org.apache.activemq.artemis.api.core.Message;
import org.apache.activemq.artemis.api.core.Pair;
import org.apache.activemq.artemis.api.core.RoutingType;
@ -69,6 +68,7 @@ import org.apache.activemq.artemis.core.server.ActiveMQServer;
import org.apache.activemq.artemis.core.server.ActiveMQServerLogger;
import org.apache.activemq.artemis.core.server.AddressQueryResult;
import org.apache.activemq.artemis.core.server.BindingQueryResult;
import org.apache.activemq.artemis.core.server.LargeServerMessage;
import org.apache.activemq.artemis.core.server.MessageReference;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.QueueQueryResult;
@ -93,6 +93,8 @@ import org.apache.activemq.artemis.utils.PrefixUtil;
import org.apache.activemq.artemis.utils.collections.TypedProperties;
import org.jboss.logging.Logger;
import static org.apache.activemq.artemis.api.core.JsonUtil.nullSafe;
/**
* Server side Session implementation
*/
@ -1309,12 +1311,34 @@ public class ServerSessionImpl implements ServerSession, FailureListener {
return send(getCurrentTransaction(), message, direct, noAutoCreateQueue);
}
private LargeServerMessage messageToLargeMessage(Message message) throws Exception {
ICoreMessage coreMessage = message.toCore();
LargeServerMessage lsm = getStorageManager().createLargeMessage(storageManager.generateID(), coreMessage);
ActiveMQBuffer buffer = coreMessage.getReadOnlyBodyBuffer();
byte[] body = new byte[buffer.readableBytes()];
buffer.readBytes(body);
lsm.addBytes(body);
lsm.releaseResources();
lsm.putLongProperty(Message.HDR_LARGE_BODY_SIZE, body.length);
return lsm;
}
@Override
public synchronized RoutingStatus send(Transaction tx,
final Message message,
Message msg,
final boolean direct,
boolean noAutoCreateQueue) throws Exception {
final Message message;
if ((msg.getEncodeSize() > storageManager.getMessageJournal().getMaxRecordSize()) && !msg.isLargeMessage()) {
message = messageToLargeMessage(msg);
} else {
message = msg;
}
server.callBrokerPlugins(server.hasBrokerPlugins() ? plugin -> plugin.beforeSend(this, tx, message, direct, noAutoCreateQueue) : null);
// If the protocol doesn't support flow control, we have no choice other than fail the communication

View File

@ -0,0 +1,63 @@
/*
* 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.tests.integration.openwire;
import javax.jms.BytesMessage;
import javax.jms.Connection;
import javax.jms.DeliveryMode;
import javax.jms.MessageProducer;
import javax.jms.Queue;
import javax.jms.Session;
import org.apache.activemq.artemis.api.core.RoutingType;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.junit.Before;
import org.junit.Test;
public class OpenWireLargeMessageTest extends BasicOpenWireTest {
public OpenWireLargeMessageTest() {
super();
}
public SimpleString lmAddress = new SimpleString("LargeMessageAddress");
@Override
@Before
public void setUp() throws Exception {
this.realStore = true;
super.setUp();
server.createQueue(lmAddress, RoutingType.ANYCAST, lmAddress, null, true, false);
}
@Test
public void testSendLargeMessage() throws Exception {
try (Connection connection = factory.createConnection()) {
Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
Queue queue = session.createQueue(lmAddress.toString());
MessageProducer producer = session.createProducer(queue);
producer.setDeliveryMode(DeliveryMode.PERSISTENT);
// Create 1MB Message
int size = 1024 * 1024;
byte[] bytes = new byte[size];
BytesMessage message = session.createBytesMessage();
message.writeBytes(bytes);
producer.send(message);
}
}
}

View File

@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.activemq.artemis.api.config.ActiveMQDefaultConfiguration;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
import org.apache.activemq.artemis.api.core.ActiveMQException;
@ -712,6 +713,11 @@ public final class ReplicationTest extends ActiveMQTestBase {
}
@Override
public long getMaxRecordSize() {
return ActiveMQDefaultConfiguration.getDefaultJournalBufferSizeAio();
}
@Override
public void appendCommitRecord(final long txID, final boolean sync) throws Exception {

View File

@ -24,6 +24,7 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.activemq.artemis.ArtemisConstants;
import org.apache.activemq.artemis.api.core.ActiveMQBuffer;
import org.apache.activemq.artemis.api.core.ActiveMQBuffers;
import org.apache.activemq.artemis.api.core.ActiveMQExceptionType;
@ -69,6 +70,11 @@ public class FakeSequentialFileFactory implements SequentialFileFactory {
return false;
}
@Override
public long getBufferSize() {
return ArtemisConstants.DEFAULT_JOURNAL_BUFFER_SIZE_AIO;
}
@Override
public int getMaxIO() {
return 1;