From 785b16bf9ef19180e7c9783442f4a125b44255e1 Mon Sep 17 00:00:00 2001 From: "Christopher L. Shannon (cshannon)" Date: Mon, 27 Apr 2015 18:24:16 +0000 Subject: [PATCH] https://issues.apache.org/jira/browse/AMQ-5748 Added a getMessageSize method to MessageStore to support retrieving the total message size of all stored messages for a destination. Added a new storeMessageSize statistic to DestinationStatistics. --- .../activemq/broker/jmx/DestinationView.java | 8 + .../broker/jmx/DestinationViewMBean.java | 8 + .../apache/activemq/broker/region/Queue.java | 1 + .../apache/activemq/broker/region/Topic.java | 1 + .../activemq/store/AbstractMessageStore.java | 21 ++ .../apache/activemq/store/MessageStore.java | 12 + .../store/MessageStoreStatistics.java | 81 ++++++ .../activemq/store/ProxyMessageStore.java | 11 + .../store/ProxyTopicMessageStore.java | 12 +- .../store/memory/MemoryMessageStore.java | 43 ++- .../management/SizeStatisticImpl.java | 17 ++ .../activemq/store/jdbc/JDBCMessageStore.java | 2 + .../store/journal/JournalMessageStore.java | 15 +- .../activemq/store/kahadb/KahaDBStore.java | 69 +++-- .../store/kahadb/MessageDatabase.java | 172 ++++++++++- .../store/kahadb/TempKahaDBStore.java | 40 +-- .../kahadb/disk/util/LocationMarshaller.java | 5 + .../activemq/leveldb/LevelDBStore.scala | 2 +- .../cursors/StoreQueueCursorOrderTest.java | 10 +- .../AbstractMessageStoreSizeStatTest.java | 266 ++++++++++++++++++ .../store/AbstractMessageStoreSizeTest.java | 98 +++++++ .../AbstractKahaDBMessageStoreSizeTest.java | 147 ++++++++++ .../KahaDBMessageStoreSizeStatTest.java | 82 ++++++ .../kahadb/KahaDBMessageStoreSizeTest.java | 46 +++ .../MultiKahaDBMessageStoreSizeStatTest.java | 134 +++++++++ .../MultiKahaDBMessageStoreSizeTest.java | 68 +++++ .../MemoryMessageStoreSizeStatTest.java | 45 +++ .../memory/MemoryMessageStoreSizeTest.java | 45 +++ .../kahadb/MessageStoreTest/version5/db-1.log | Bin 0 -> 524288 bytes .../kahadb/MessageStoreTest/version5/db.data | Bin 0 -> 32768 bytes .../kahadb/MessageStoreTest/version5/db.redo | Bin 0 -> 32824 bytes .../version5/queue#3a#2f#2fTest/db-1.log | Bin 0 -> 524288 bytes .../version5/queue#3a#2f#2fTest/db.data | Bin 0 -> 32768 bytes .../version5/queue#3a#2f#2fTest/db.redo | Bin 0 -> 32824 bytes 34 files changed, 1382 insertions(+), 79 deletions(-) create mode 100644 activemq-broker/src/main/java/org/apache/activemq/store/MessageStoreStatistics.java create mode 100644 activemq-unit-tests/src/test/java/org/apache/activemq/store/AbstractMessageStoreSizeStatTest.java create mode 100644 activemq-unit-tests/src/test/java/org/apache/activemq/store/AbstractMessageStoreSizeTest.java create mode 100644 activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/AbstractKahaDBMessageStoreSizeTest.java create mode 100644 activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/KahaDBMessageStoreSizeStatTest.java create mode 100644 activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/KahaDBMessageStoreSizeTest.java create mode 100644 activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/MultiKahaDBMessageStoreSizeStatTest.java create mode 100644 activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/MultiKahaDBMessageStoreSizeTest.java create mode 100644 activemq-unit-tests/src/test/java/org/apache/activemq/store/memory/MemoryMessageStoreSizeStatTest.java create mode 100644 activemq-unit-tests/src/test/java/org/apache/activemq/store/memory/MemoryMessageStoreSizeTest.java create mode 100644 activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MessageStoreTest/version5/db-1.log create mode 100644 activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MessageStoreTest/version5/db.data create mode 100644 activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MessageStoreTest/version5/db.redo create mode 100644 activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MultiKahaMessageStoreTest/version5/queue#3a#2f#2fTest/db-1.log create mode 100644 activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MultiKahaMessageStoreTest/version5/queue#3a#2f#2fTest/db.data create mode 100644 activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MultiKahaMessageStoreTest/version5/queue#3a#2f#2fTest/db.redo diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/jmx/DestinationView.java b/activemq-broker/src/main/java/org/apache/activemq/broker/jmx/DestinationView.java index b3bf86995d..3e51a49d4e 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/jmx/DestinationView.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/jmx/DestinationView.java @@ -38,6 +38,7 @@ import javax.management.openmbean.OpenDataException; import javax.management.openmbean.TabularData; import javax.management.openmbean.TabularDataSupport; import javax.management.openmbean.TabularType; + import org.apache.activemq.ActiveMQConnectionFactory; import org.apache.activemq.broker.jmx.OpenTypeSupport.OpenTypeFactory; import org.apache.activemq.broker.region.Destination; @@ -51,6 +52,7 @@ import org.apache.activemq.command.Message; import org.apache.activemq.filter.BooleanExpression; import org.apache.activemq.filter.MessageEvaluationContext; import org.apache.activemq.selector.SelectorParser; +import org.apache.activemq.store.MessageStore; import org.apache.activemq.util.URISupport; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -119,6 +121,12 @@ public class DestinationView implements DestinationViewMBean { return destination.getDestinationStatistics().getMessages().getCount(); } + @Override + public long getStoreMessageSize() { + MessageStore messageStore = destination.getMessageStore(); + return messageStore != null ? messageStore.getMessageStoreStatistics().getMessageSize().getTotalSize() : 0; + } + public long getMessagesCached() { return destination.getDestinationStatistics().getMessagesCached().getCount(); } diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/jmx/DestinationViewMBean.java b/activemq-broker/src/main/java/org/apache/activemq/broker/jmx/DestinationViewMBean.java index 60340ffe81..aedc15d8f2 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/jmx/DestinationViewMBean.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/jmx/DestinationViewMBean.java @@ -121,6 +121,14 @@ public interface DestinationViewMBean { @MBeanInfo("Number of messages in the destination which are yet to be consumed. Potentially dispatched but unacknowledged.") long getQueueSize(); + /** + * Returns the memory size of all messages in this destination's store + * + * @return Returns the memory size of all messages in this destination's store + */ + @MBeanInfo("The memory size of all messages in this destination's store.") + long getStoreMessageSize(); + /** * @return An array of all the messages in the destination's queue. */ diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/Queue.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/Queue.java index af61e1919e..c9823e1c22 100755 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/Queue.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/Queue.java @@ -375,6 +375,7 @@ public class Queue extends BaseDestination implements Task, UsageListener, Index messages.setMaxProducersToAudit(getMaxProducersToAudit()); messages.setUseCache(isUseCache()); messages.setMemoryUsageHighWaterMark(getCursorMemoryHighWaterMark()); + store.start(); final int messageCount = store.getMessageCount(); if (messageCount > 0 && messages.isRecoveryRequired()) { BatchMessageRecoveryListener listener = new BatchMessageRecoveryListener(messageCount); diff --git a/activemq-broker/src/main/java/org/apache/activemq/broker/region/Topic.java b/activemq-broker/src/main/java/org/apache/activemq/broker/region/Topic.java index bda000b696..61c62ce963 100755 --- a/activemq-broker/src/main/java/org/apache/activemq/broker/region/Topic.java +++ b/activemq-broker/src/main/java/org/apache/activemq/broker/region/Topic.java @@ -105,6 +105,7 @@ public class Topic extends BaseDestination implements Task { // misleading metrics. // int messageCount = store.getMessageCount(); // destinationStatistics.getMessages().setCount(messageCount); + store.start(); } } diff --git a/activemq-broker/src/main/java/org/apache/activemq/store/AbstractMessageStore.java b/activemq-broker/src/main/java/org/apache/activemq/store/AbstractMessageStore.java index faa6c1fbcf..413f958c01 100644 --- a/activemq-broker/src/main/java/org/apache/activemq/store/AbstractMessageStore.java +++ b/activemq-broker/src/main/java/org/apache/activemq/store/AbstractMessageStore.java @@ -30,6 +30,7 @@ abstract public class AbstractMessageStore implements MessageStore { protected final ActiveMQDestination destination; protected boolean prioritizedMessages; protected IndexListener indexListener; + protected final MessageStoreStatistics messageStoreStatistics = new MessageStoreStatistics(); public AbstractMessageStore(ActiveMQDestination destination) { this.destination = destination; @@ -41,6 +42,7 @@ abstract public class AbstractMessageStore implements MessageStore { @Override public void start() throws Exception { + recoverMessageStoreStatistics(); } @Override @@ -132,4 +134,23 @@ abstract public class AbstractMessageStore implements MessageStore { static { FUTURE = new InlineListenableFuture(); } + + @Override + public int getMessageCount() throws IOException { + return (int) getMessageStoreStatistics().getMessageCount().getCount(); + } + + @Override + public long getMessageSize() throws IOException { + return getMessageStoreStatistics().getMessageSize().getTotalSize(); + } + + @Override + public MessageStoreStatistics getMessageStoreStatistics() { + return messageStoreStatistics; + } + + protected void recoverMessageStoreStatistics() throws IOException { + + } } diff --git a/activemq-broker/src/main/java/org/apache/activemq/store/MessageStore.java b/activemq-broker/src/main/java/org/apache/activemq/store/MessageStore.java index 4cc472e0c7..aee619a27a 100755 --- a/activemq-broker/src/main/java/org/apache/activemq/store/MessageStore.java +++ b/activemq-broker/src/main/java/org/apache/activemq/store/MessageStore.java @@ -158,6 +158,18 @@ public interface MessageStore extends Service { */ int getMessageCount() throws IOException; + /** + * @return the size of the messages ready to deliver + * @throws IOException + */ + long getMessageSize() throws IOException; + + + /** + * @return The statistics bean for this message store + */ + MessageStoreStatistics getMessageStoreStatistics(); + /** * A hint to the Store to reset any batching state for the Destination * diff --git a/activemq-broker/src/main/java/org/apache/activemq/store/MessageStoreStatistics.java b/activemq-broker/src/main/java/org/apache/activemq/store/MessageStoreStatistics.java new file mode 100644 index 0000000000..0a2b021610 --- /dev/null +++ b/activemq-broker/src/main/java/org/apache/activemq/store/MessageStoreStatistics.java @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.activemq.store; + +import org.apache.activemq.management.CountStatisticImpl; +import org.apache.activemq.management.SizeStatisticImpl; +import org.apache.activemq.management.StatsImpl; + +/** + * The J2EE Statistics for a Message Sore + */ +public class MessageStoreStatistics extends StatsImpl { + + protected CountStatisticImpl messageCount; + protected SizeStatisticImpl messageSize; + + + public MessageStoreStatistics() { + this(true); + } + + public MessageStoreStatistics(boolean enabled) { + + messageCount = new CountStatisticImpl("messageCount", "The number of messages in the store passing through the destination"); + messageSize = new SizeStatisticImpl("messageSize","Size of messages in the store passing through the destination"); + + addStatistic("messageCount", messageCount); + addStatistic("messageSize", messageSize); + + this.setEnabled(enabled); + } + + + public CountStatisticImpl getMessageCount() { + return messageCount; + } + + public SizeStatisticImpl getMessageSize() { + return messageSize; + } + + public void reset() { + if (this.isDoReset()) { + super.reset(); + messageCount.reset(); + messageSize.reset(); + } + } + + public void setEnabled(boolean enabled) { + super.setEnabled(enabled); + messageCount.setEnabled(enabled); + messageSize.setEnabled(enabled); + } + + public void setParent(MessageStoreStatistics parent) { + if (parent != null) { + messageCount.setParent(parent.messageCount); + messageSize.setParent(parent.messageSize); + } else { + messageCount.setParent(null); + messageSize.setParent(null); + } + } + +} diff --git a/activemq-broker/src/main/java/org/apache/activemq/store/ProxyMessageStore.java b/activemq-broker/src/main/java/org/apache/activemq/store/ProxyMessageStore.java index c9b2060b54..cd319a65d3 100755 --- a/activemq-broker/src/main/java/org/apache/activemq/store/ProxyMessageStore.java +++ b/activemq-broker/src/main/java/org/apache/activemq/store/ProxyMessageStore.java @@ -100,6 +100,11 @@ public class ProxyMessageStore implements MessageStore { return delegate.getMessageCount(); } + @Override + public long getMessageSize() throws IOException { + return delegate.getMessageSize(); + } + @Override public void recoverNextMessages(int maxReturned, MessageRecoveryListener listener) throws Exception { delegate.recoverNextMessages(maxReturned, listener); @@ -169,4 +174,10 @@ public class ProxyMessageStore implements MessageStore { public String toString() { return delegate.toString(); } + + @Override + public MessageStoreStatistics getMessageStoreStatistics() { + return delegate.getMessageStoreStatistics(); + } + } diff --git a/activemq-broker/src/main/java/org/apache/activemq/store/ProxyTopicMessageStore.java b/activemq-broker/src/main/java/org/apache/activemq/store/ProxyTopicMessageStore.java index 0f47f61b8f..5c591583b0 100755 --- a/activemq-broker/src/main/java/org/apache/activemq/store/ProxyTopicMessageStore.java +++ b/activemq-broker/src/main/java/org/apache/activemq/store/ProxyTopicMessageStore.java @@ -17,7 +17,6 @@ package org.apache.activemq.store; import java.io.IOException; -import java.util.concurrent.Future; import org.apache.activemq.broker.ConnectionContext; import org.apache.activemq.command.ActiveMQDestination; @@ -145,6 +144,11 @@ public class ProxyTopicMessageStore implements TopicMessageStore { return delegate.getMessageCount(); } + @Override + public long getMessageSize() throws IOException { + return delegate.getMessageSize(); + } + @Override public void recoverNextMessages(int maxReturned, MessageRecoveryListener listener) throws Exception { delegate.recoverNextMessages(maxReturned, listener); @@ -213,4 +217,10 @@ public class ProxyTopicMessageStore implements TopicMessageStore { public void registerIndexListener(IndexListener indexListener) { delegate.registerIndexListener(indexListener); } + + @Override + public MessageStoreStatistics getMessageStoreStatistics() { + return delegate.getMessageStoreStatistics(); + } + } diff --git a/activemq-broker/src/main/java/org/apache/activemq/store/memory/MemoryMessageStore.java b/activemq-broker/src/main/java/org/apache/activemq/store/memory/MemoryMessageStore.java index 7cdaa78b0f..e71dab834d 100755 --- a/activemq-broker/src/main/java/org/apache/activemq/store/memory/MemoryMessageStore.java +++ b/activemq-broker/src/main/java/org/apache/activemq/store/memory/MemoryMessageStore.java @@ -35,8 +35,8 @@ import org.apache.activemq.store.AbstractMessageStore; /** * An implementation of {@link org.apache.activemq.store.MessageStore} which * uses a - * - * + * + * */ public class MemoryMessageStore extends AbstractMessageStore { @@ -56,6 +56,8 @@ public class MemoryMessageStore extends AbstractMessageStore { public synchronized void addMessage(ConnectionContext context, Message message) throws IOException { synchronized (messageTable) { messageTable.put(message.getMessageId(), message); + getMessageStoreStatistics().getMessageCount().increment(); + getMessageStoreStatistics().getMessageSize().addSize(message.getSize()); } message.incrementReferenceCount(); message.getMessageId().setFutureOrSequenceLong(sequenceId++); @@ -93,6 +95,8 @@ public class MemoryMessageStore extends AbstractMessageStore { if ((lastBatchId != null && lastBatchId.equals(msgId)) || messageTable.isEmpty()) { lastBatchId = null; } + getMessageStoreStatistics().getMessageCount().decrement(); + getMessageStoreStatistics().getMessageSize().addSize(-removed.getSize()); } } @@ -114,20 +118,17 @@ public class MemoryMessageStore extends AbstractMessageStore { public void removeAllMessages(ConnectionContext context) throws IOException { synchronized (messageTable) { messageTable.clear(); + getMessageStoreStatistics().reset(); } } public void delete() { synchronized (messageTable) { messageTable.clear(); + getMessageStoreStatistics().reset(); } } - - public int getMessageCount() { - return messageTable.size(); - } - public void recoverNextMessages(int maxReturned, MessageRecoveryListener listener) throws Exception { synchronized (messageTable) { boolean pastLackBatch = lastBatchId == null; @@ -161,8 +162,34 @@ public class MemoryMessageStore extends AbstractMessageStore { public void updateMessage(Message message) { synchronized (messageTable) { + Message original = messageTable.get(message.getMessageId()); + + //if can't be found then increment count, else remove old size + if (original == null) { + getMessageStoreStatistics().getMessageCount().increment(); + } else { + getMessageStoreStatistics().getMessageSize().addSize(-original.getSize()); + } messageTable.put(message.getMessageId(), message); + getMessageStoreStatistics().getMessageSize().addSize(message.getSize()); } } - + + @Override + public void recoverMessageStoreStatistics() throws IOException { + synchronized (messageTable) { + long size = 0; + int count = 0; + for (Iterator iter = messageTable.values().iterator(); iter + .hasNext();) { + Message msg = iter.next(); + size += msg.getSize(); + } + + getMessageStoreStatistics().reset(); + getMessageStoreStatistics().getMessageCount().setCount(count); + getMessageStoreStatistics().getMessageSize().setTotalSize(size); + } + } + } diff --git a/activemq-client/src/main/java/org/apache/activemq/management/SizeStatisticImpl.java b/activemq-client/src/main/java/org/apache/activemq/management/SizeStatisticImpl.java index 1cf0058963..e2bc033e21 100644 --- a/activemq-client/src/main/java/org/apache/activemq/management/SizeStatisticImpl.java +++ b/activemq-client/src/main/java/org/apache/activemq/management/SizeStatisticImpl.java @@ -67,6 +67,23 @@ public class SizeStatisticImpl extends StatisticImpl { } } + /** + * Reset the total size to the new value + * + * @param size + */ + public synchronized void setTotalSize(long size) { + count++; + totalSize = size; + if (size > maxSize) { + maxSize = size; + } + if (size < minSize || minSize == 0) { + minSize = size; + } + updateSampleTime(); + } + /** * @return the maximum size of any step */ diff --git a/activemq-jdbc-store/src/main/java/org/apache/activemq/store/jdbc/JDBCMessageStore.java b/activemq-jdbc-store/src/main/java/org/apache/activemq/store/jdbc/JDBCMessageStore.java index 4674d7a6be..ac4e8cef8d 100755 --- a/activemq-jdbc-store/src/main/java/org/apache/activemq/store/jdbc/JDBCMessageStore.java +++ b/activemq-jdbc-store/src/main/java/org/apache/activemq/store/jdbc/JDBCMessageStore.java @@ -304,6 +304,7 @@ public class JDBCMessageStore extends AbstractMessageStore { } } + @Override public int getMessageCount() throws IOException { int result = 0; TransactionContext c = persistenceAdapter.getTransactionContext(); @@ -401,4 +402,5 @@ public class JDBCMessageStore extends AbstractMessageStore { public String toString() { return destination.getPhysicalName() + ",pendingSize:" + pendingAdditions.size(); } + } diff --git a/activemq-jdbc-store/src/main/java/org/apache/activemq/store/journal/JournalMessageStore.java b/activemq-jdbc-store/src/main/java/org/apache/activemq/store/journal/JournalMessageStore.java index 2d44769a61..7ec10c4d9a 100755 --- a/activemq-jdbc-store/src/main/java/org/apache/activemq/store/journal/JournalMessageStore.java +++ b/activemq-jdbc-store/src/main/java/org/apache/activemq/store/journal/JournalMessageStore.java @@ -48,8 +48,8 @@ import org.slf4j.LoggerFactory; /** * A MessageStore that uses a Journal to store it's messages. - * - * + * + * */ public class JournalMessageStore extends AbstractMessageStore { @@ -79,7 +79,7 @@ public class JournalMessageStore extends AbstractMessageStore { this.transactionTemplate = new TransactionTemplate(adapter, new ConnectionContext(new NonCachedMessageEvaluationContext())); } - + public void setMemoryUsage(MemoryUsage memoryUsage) { this.memoryUsage=memoryUsage; longTermStore.setMemoryUsage(memoryUsage); @@ -323,7 +323,7 @@ public class JournalMessageStore extends AbstractMessageStore { } /** - * + * */ public Message getMessage(MessageId identity) throws IOException { Message answer = null; @@ -348,7 +348,7 @@ public class JournalMessageStore extends AbstractMessageStore { * Replays the checkpointStore first as those messages are the oldest ones, * then messages are replayed from the transaction log and then the cache is * updated. - * + * * @param listener * @throws Exception */ @@ -404,6 +404,11 @@ public class JournalMessageStore extends AbstractMessageStore { return longTermStore.getMessageCount(); } + public long getMessageSize() throws IOException { + peristenceAdapter.checkpoint(true, true); + return longTermStore.getMessageSize(); + } + public void recoverNextMessages(int maxReturned, MessageRecoveryListener listener) throws Exception { peristenceAdapter.checkpoint(true, true); longTermStore.recoverNextMessages(maxReturned, listener); diff --git a/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/KahaDBStore.java b/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/KahaDBStore.java index 8ceef360a4..44f93a622a 100644 --- a/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/KahaDBStore.java +++ b/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/KahaDBStore.java @@ -61,6 +61,7 @@ import org.apache.activemq.store.IndexListener; import org.apache.activemq.store.ListenableFuture; import org.apache.activemq.store.MessageRecoveryListener; import org.apache.activemq.store.MessageStore; +import org.apache.activemq.store.MessageStoreStatistics; import org.apache.activemq.store.PersistenceAdapter; import org.apache.activemq.store.TopicMessageStore; import org.apache.activemq.store.TransactionIdTransformer; @@ -503,34 +504,6 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter { return loadMessage(location); } - @Override - public int getMessageCount() throws IOException { - try { - lockAsyncJobQueue(); - indexLock.writeLock().lock(); - try { - return pageFile.tx().execute(new Transaction.CallableClosure() { - @Override - public Integer execute(Transaction tx) throws IOException { - // Iterate through all index entries to get a count - // of messages in the destination. - StoredDestination sd = getStoredDestination(dest, tx); - int rc = 0; - for (Iterator> iterator = sd.locationIndex.iterator(tx); iterator.hasNext();) { - iterator.next(); - rc++; - } - return rc; - } - }); - } finally { - indexLock.writeLock().unlock(); - } - } finally { - unlockAsyncJobQueue(); - } - } - @Override public boolean isEmpty() throws IOException { indexLock.writeLock().lock(); @@ -716,6 +689,38 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter { public String toString(){ return "permits:" + this.localDestinationSemaphore.availablePermits() + ",sd=" + storedDestinations.get(key(dest)); } + + @Override + protected void recoverMessageStoreStatistics() throws IOException { + try { + MessageStoreStatistics recoveredStatistics; + lockAsyncJobQueue(); + indexLock.writeLock().lock(); + try { + recoveredStatistics = pageFile.tx().execute(new Transaction.CallableClosure() { + @Override + public MessageStoreStatistics execute(Transaction tx) throws IOException { + MessageStoreStatistics statistics = new MessageStoreStatistics(); + + // Iterate through all index entries to get the size of each message + StoredDestination sd = getStoredDestination(dest, tx); + for (Iterator> iterator = sd.locationIndex.iterator(tx); iterator.hasNext();) { + int locationSize = iterator.next().getKey().getSize(); + statistics.getMessageCount().increment(); + statistics.getMessageSize().addSize(locationSize > 0 ? locationSize : 0); + } + return statistics; + } + }); + getMessageStoreStatistics().getMessageCount().setCount(recoveredStatistics.getMessageCount().getCount()); + getMessageStoreStatistics().getMessageSize().setTotalSize(recoveredStatistics.getMessageSize().getTotalSize()); + } finally { + indexLock.writeLock().unlock(); + } + } finally { + unlockAsyncJobQueue(); + } + } } class KahaDBTopicMessageStore extends KahaDBMessageStore implements TopicMessageStore { @@ -993,12 +998,16 @@ public class KahaDBStore extends MessageDatabase implements PersistenceAdapter { @Override public MessageStore createQueueMessageStore(ActiveMQQueue destination) throws IOException { - return this.transactionStore.proxy(new KahaDBMessageStore(destination)); + MessageStore store = this.transactionStore.proxy(new KahaDBMessageStore(destination)); + storeCache.put(key(convert(destination)), store); + return store; } @Override public TopicMessageStore createTopicMessageStore(ActiveMQTopic destination) throws IOException { - return this.transactionStore.proxy(new KahaDBTopicMessageStore(destination)); + TopicMessageStore store = this.transactionStore.proxy(new KahaDBTopicMessageStore(destination)); + storeCache.put(key(convert(destination)), store); + return store; } /** diff --git a/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/MessageDatabase.java b/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/MessageDatabase.java index ef8fe0a396..e35619e4f4 100644 --- a/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/MessageDatabase.java +++ b/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/MessageDatabase.java @@ -46,6 +46,7 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeMap; import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -53,10 +54,15 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.activemq.ActiveMQMessageAuditNoSync; import org.apache.activemq.broker.BrokerService; import org.apache.activemq.broker.BrokerServiceAware; +import org.apache.activemq.broker.region.Destination; +import org.apache.activemq.broker.region.Queue; +import org.apache.activemq.broker.region.Topic; import org.apache.activemq.command.MessageAck; import org.apache.activemq.command.TransactionId; import org.apache.activemq.openwire.OpenWireFormat; import org.apache.activemq.protobuf.Buffer; +import org.apache.activemq.store.MessageStore; +import org.apache.activemq.store.MessageStoreStatistics; import org.apache.activemq.store.kahadb.data.KahaAckMessageFileMapCommand; import org.apache.activemq.store.kahadb.data.KahaAddMessageCommand; import org.apache.activemq.store.kahadb.data.KahaCommitCommand; @@ -113,7 +119,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe static final int OPEN_STATE = 2; static final long NOT_ACKED = -1; - static final int VERSION = 5; + static final int VERSION = 6; protected class Metadata { protected Page page; @@ -738,7 +744,8 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe long undoCounter=0; // Go through all the destinations to see if they have messages past the lastAppendLocation - for (StoredDestination sd : storedDestinations.values()) { + for (String key : storedDestinations.keySet()) { + StoredDestination sd = storedDestinations.get(key); final ArrayList matches = new ArrayList(); // Find all the Locations that are >= than the last Append Location. @@ -755,6 +762,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe sd.messageIdIndex.remove(tx, keys.messageId); metadata.producerSequenceIdTracker.rollback(keys.messageId); undoCounter++; + decrementAndSubSizeToStoreStat(key, keys.location.getSize()); // TODO: do we need to modify the ack positions for the pub sub case? } } @@ -858,6 +866,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe sd.messageIdIndex.remove(tx, keys.messageId); LOG.info("[" + sdEntry.getKey() + "] dropped: " + keys.messageId + " at corrupt location: " + keys.location); undoCounter++; + decrementAndSubSizeToStoreStat(sdEntry.getKey(), keys.location.getSize()); // TODO: do we need to modify the ack positions for the pub sub case? } } else { @@ -1312,6 +1321,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe if (previous == null) { previous = sd.messageIdIndex.put(tx, command.getMessageId(), id); if (previous == null) { + incrementAndAddSizeToStoreStat(command.getDestination(), location.getSize()); sd.orderIndex.put(tx, priority, id, new MessageKeys(command.getMessageId(), location)); if (sd.subscriptions != null && !sd.subscriptions.isEmpty(tx)) { addAckLocationForNewMessage(tx, sd, id); @@ -1337,7 +1347,8 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe } // record this id in any event, initial send or recovery metadata.producerSequenceIdTracker.isDuplicate(command.getMessageId()); - return id; + + return id; } void trackPendingAdd(KahaDestination destination, Long seq) { @@ -1367,9 +1378,11 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe new MessageKeys(command.getMessageId(), location) ); sd.locationIndex.put(tx, location, id); + incrementAndAddSizeToStoreStat(command.getDestination(), location.getSize()); // on first update previous is original location, on recovery/replay it may be the updated location if(previousKeys != null && !previousKeys.location.equals(location)) { sd.locationIndex.remove(tx, previousKeys.location); + decrementAndSubSizeToStoreStat(command.getDestination(), previousKeys.location.getSize()); } metadata.lastUpdate = location; } else { @@ -1387,6 +1400,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe MessageKeys keys = sd.orderIndex.remove(tx, sequenceId); if (keys != null) { sd.locationIndex.remove(tx, keys.location); + decrementAndSubSizeToStoreStat(command.getDestination(), keys.location.getSize()); recordAckMessageReferenceLocation(ackLocation, keys.location); metadata.lastUpdate = ackLocation; } else if (LOG.isDebugEnabled()) { @@ -1414,7 +1428,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe recordAckMessageReferenceLocation(ackLocation, keys.location); } // The following method handles deleting un-referenced messages. - removeAckLocation(tx, sd, subscriptionKey, sequence); + removeAckLocation(command, tx, sd, subscriptionKey, sequence); metadata.lastUpdate = ackLocation; } else if (LOG.isDebugEnabled()) { LOG.debug("no message sequence exists for id: " + command.getMessageId() + " and sub: " + command.getSubscriptionKey()); @@ -1470,6 +1484,8 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe String key = key(command.getDestination()); storedDestinations.remove(key); metadata.destinations.remove(tx, key); + clearStoreStats(command.getDestination()); + storeCache.remove(key(command.getDestination())); } void updateIndex(Transaction tx, KahaSubscriptionCommand command, Location location) throws IOException { @@ -1494,13 +1510,14 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe sd.subLocations.remove(tx, subscriptionKey); sd.subscriptionAcks.remove(tx, subscriptionKey); sd.subscriptionCache.remove(subscriptionKey); - removeAckLocationsForSub(tx, sd, subscriptionKey); + removeAckLocationsForSub(command, tx, sd, subscriptionKey); if (sd.subscriptions.isEmpty(tx)) { // remove the stored destination KahaRemoveDestinationCommand removeDestinationCommand = new KahaRemoveDestinationCommand(); removeDestinationCommand.setDestination(command.getDestination()); updateIndex(tx, removeDestinationCommand, null); + clearStoreStats(command.getDestination()); } } } @@ -1879,6 +1896,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe } } + class StoredDestination { MessageOrderIndex orderIndex = new MessageOrderIndex(); @@ -1912,6 +1930,8 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe protected class StoredDestinationMarshaller extends VariableMarshaller { + final MessageKeysMarshaller messageKeysMarshaller = new MessageKeysMarshaller(); + @Override public StoredDestination readPayload(final DataInput dataIn) throws IOException { final StoredDestination value = new StoredDestination(); @@ -1996,12 +2016,12 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe public void execute(Transaction tx) throws IOException { value.orderIndex.lowPriorityIndex = new BTreeIndex(pageFile, tx.allocate()); value.orderIndex.lowPriorityIndex.setKeyMarshaller(LongMarshaller.INSTANCE); - value.orderIndex.lowPriorityIndex.setValueMarshaller(MessageKeysMarshaller.INSTANCE); + value.orderIndex.lowPriorityIndex.setValueMarshaller(messageKeysMarshaller); value.orderIndex.lowPriorityIndex.load(tx); value.orderIndex.highPriorityIndex = new BTreeIndex(pageFile, tx.allocate()); value.orderIndex.highPriorityIndex.setKeyMarshaller(LongMarshaller.INSTANCE); - value.orderIndex.highPriorityIndex.setValueMarshaller(MessageKeysMarshaller.INSTANCE); + value.orderIndex.highPriorityIndex.setValueMarshaller(messageKeysMarshaller); value.orderIndex.highPriorityIndex.load(tx); } }); @@ -2100,7 +2120,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe // Figure out the next key using the last entry in the destination. rc.orderIndex.configureLast(tx); - rc.locationIndex.setKeyMarshaller(org.apache.activemq.store.kahadb.disk.util.LocationMarshaller.INSTANCE); + rc.locationIndex.setKeyMarshaller(new LocationSizeMarshaller()); rc.locationIndex.setValueMarshaller(LongMarshaller.INSTANCE); rc.locationIndex.load(tx); @@ -2202,6 +2222,133 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe return rc; } + /** + * Clear the counter for the destination, if one exists. + * + * @param kahaDestination + */ + protected void clearStoreStats(KahaDestination kahaDestination) { + MessageStoreStatistics storeStats = getStoreStats(key(kahaDestination)); + if (storeStats != null) { + storeStats.reset(); + } + } + + /** + * Update MessageStoreStatistics + * + * @param kahaDestination + * @param size + */ + protected void incrementAndAddSizeToStoreStat(KahaDestination kahaDestination, long size) { + incrementAndAddSizeToStoreStat(key(kahaDestination), size); + } + + protected void incrementAndAddSizeToStoreStat(String kahaDestKey, long size) { + MessageStoreStatistics storeStats = getStoreStats(kahaDestKey); + if (storeStats != null) { + storeStats.getMessageCount().increment(); + if (size > 0) { + storeStats.getMessageSize().addSize(size); + } + } + } + + protected void decrementAndSubSizeToStoreStat(KahaDestination kahaDestination, long size) { + decrementAndSubSizeToStoreStat(key(kahaDestination), size); + } + + protected void decrementAndSubSizeToStoreStat(String kahaDestKey, long size) { + MessageStoreStatistics storeStats = getStoreStats(kahaDestKey); + if (storeStats != null) { + storeStats.getMessageCount().decrement(); + if (size > 0) { + storeStats.getMessageSize().addSize(-size); + } + } + } + + /** + * This is a map to cache DestinationStatistics for a specific + * KahaDestination key + */ + protected final Map storeCache = + new ConcurrentHashMap(); + + /** + * Locate the storeMessageSize counter for this KahaDestination + * @param kahaDestination + * @return + */ + protected MessageStoreStatistics getStoreStats(String kahaDestKey) { + MessageStoreStatistics storeStats = null; + try { + MessageStore messageStore = storeCache.get(kahaDestKey); + if (messageStore != null) { + storeStats = messageStore.getMessageStoreStatistics(); + } + } catch (Exception e1) { + LOG.error("Getting size counter of destination failed", e1); + } + + return storeStats; + } + + /** + * Determine whether this Destination matches the DestinationType + * + * @param destination + * @param type + * @return + */ + protected boolean matchType(Destination destination, + KahaDestination.DestinationType type) { + if (destination instanceof Topic + && type.equals(KahaDestination.DestinationType.TOPIC)) { + return true; + } else if (destination instanceof Queue + && type.equals(KahaDestination.DestinationType.QUEUE)) { + return true; + } + return false; + } + + class LocationSizeMarshaller implements Marshaller { + + public LocationSizeMarshaller() { + + } + + public Location readPayload(DataInput dataIn) throws IOException { + Location rc = new Location(); + rc.setDataFileId(dataIn.readInt()); + rc.setOffset(dataIn.readInt()); + if (metadata.version >= 6) { + rc.setSize(dataIn.readInt()); + } + return rc; + } + + public void writePayload(Location object, DataOutput dataOut) + throws IOException { + dataOut.writeInt(object.getDataFileId()); + dataOut.writeInt(object.getOffset()); + dataOut.writeInt(object.getSize()); + } + + public int getFixedSize() { + return 12; + } + + public Location deepCopy(Location source) { + return new Location(source); + } + + public boolean isDeepCopySupported() { + return true; + } + } + private void addAckLocation(Transaction tx, StoredDestination sd, Long messageSequence, String subscriptionKey) throws IOException { SequenceSet sequences = sd.ackPositions.get(tx, subscriptionKey); if (sequences == null) { @@ -2269,7 +2416,8 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe } } - private void removeAckLocationsForSub(Transaction tx, StoredDestination sd, String subscriptionKey) throws IOException { + private void removeAckLocationsForSub(KahaSubscriptionCommand command, + Transaction tx, StoredDestination sd, String subscriptionKey) throws IOException { if (!sd.ackPositions.isEmpty(tx)) { SequenceSet sequences = sd.ackPositions.remove(tx, subscriptionKey); if (sequences == null || sequences.isEmpty()) { @@ -2302,6 +2450,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe sd.locationIndex.remove(tx, entry.getValue().location); sd.messageIdIndex.remove(tx, entry.getValue().messageId); sd.orderIndex.remove(tx, entry.getKey()); + decrementAndSubSizeToStoreStat(command.getDestination(), entry.getValue().location.getSize()); } } } @@ -2314,7 +2463,9 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe * @param messageSequence * @throws IOException */ - private void removeAckLocation(Transaction tx, StoredDestination sd, String subscriptionKey, Long messageSequence) throws IOException { + private void removeAckLocation(KahaRemoveMessageCommand command, + Transaction tx, StoredDestination sd, String subscriptionKey, + Long messageSequence) throws IOException { // Remove the sub from the previous location set.. if (messageSequence != null) { SequenceSet range = sd.ackPositions.get(tx, subscriptionKey); @@ -2347,6 +2498,7 @@ public abstract class MessageDatabase extends ServiceSupport implements BrokerSe sd.locationIndex.remove(tx, entry.getValue().location); sd.messageIdIndex.remove(tx, entry.getValue().messageId); sd.orderIndex.remove(tx, entry.getKey()); + decrementAndSubSizeToStoreStat(command.getDestination(), entry.getValue().location.getSize()); } } } diff --git a/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/TempKahaDBStore.java b/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/TempKahaDBStore.java index 45e35c6f07..04d74b6357 100644 --- a/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/TempKahaDBStore.java +++ b/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/TempKahaDBStore.java @@ -197,25 +197,6 @@ public class TempKahaDBStore extends TempMessageDatabase implements PersistenceA return msg; } - @Override - public int getMessageCount() throws IOException { - synchronized(indexMutex) { - return pageFile.tx().execute(new Transaction.CallableClosure(){ - @Override - public Integer execute(Transaction tx) throws IOException { - // Iterate through all index entries to get a count of messages in the destination. - StoredDestination sd = getStoredDestination(dest, tx); - int rc=0; - for (Iterator> iterator = sd.messageIdIndex.iterator(tx); iterator.hasNext();) { - iterator.next(); - rc++; - } - return rc; - } - }); - } - } - @Override public void recover(final MessageRecoveryListener listener) throws Exception { synchronized(indexMutex) { @@ -297,6 +278,27 @@ public class TempKahaDBStore extends TempMessageDatabase implements PersistenceA public void stop() throws Exception { } + @Override + public void recoverMessageStoreStatistics() throws IOException { + int count = 0; + synchronized(indexMutex) { + count = pageFile.tx().execute(new Transaction.CallableClosure(){ + @Override + public Integer execute(Transaction tx) throws IOException { + // Iterate through all index entries to get a count of messages in the destination. + StoredDestination sd = getStoredDestination(dest, tx); + int rc=0; + for (Iterator> iterator = sd.messageIdIndex.iterator(tx); iterator.hasNext();) { + iterator.next(); + rc++; + } + return rc; + } + }); + } + getMessageStoreStatistics().getMessageCount().setCount(count); + } + } class KahaDBTopicMessageStore extends KahaDBMessageStore implements TopicMessageStore { diff --git a/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/disk/util/LocationMarshaller.java b/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/disk/util/LocationMarshaller.java index 7826a0ba8b..e859f9c672 100644 --- a/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/disk/util/LocationMarshaller.java +++ b/activemq-kahadb-store/src/main/java/org/apache/activemq/store/kahadb/disk/util/LocationMarshaller.java @@ -22,8 +22,13 @@ import java.io.IOException; import org.apache.activemq.store.kahadb.disk.journal.Location; public class LocationMarshaller implements Marshaller { + public final static LocationMarshaller INSTANCE = new LocationMarshaller(); + public LocationMarshaller () { + + } + public Location readPayload(DataInput dataIn) throws IOException { Location rc = new Location(); rc.setDataFileId(dataIn.readInt()); diff --git a/activemq-leveldb-store/src/main/scala/org/apache/activemq/leveldb/LevelDBStore.scala b/activemq-leveldb-store/src/main/scala/org/apache/activemq/leveldb/LevelDBStore.scala index 49e8cfab51..7c2d327519 100644 --- a/activemq-leveldb-store/src/main/scala/org/apache/activemq/leveldb/LevelDBStore.scala +++ b/activemq-leveldb-store/src/main/scala/org/apache/activemq/leveldb/LevelDBStore.scala @@ -834,7 +834,7 @@ class LevelDBStore extends LockableServiceSupport with BrokerServiceAware with P cursorPosition = cursorResetPosition } - def getMessageCount: Int = { + override def getMessageCount: Int = { return db.collectionSize(key).toInt } diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/broker/region/cursors/StoreQueueCursorOrderTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/broker/region/cursors/StoreQueueCursorOrderTest.java index f8fab10a13..90b8428640 100644 --- a/activemq-unit-tests/src/test/java/org/apache/activemq/broker/region/cursors/StoreQueueCursorOrderTest.java +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/broker/region/cursors/StoreQueueCursorOrderTest.java @@ -490,11 +490,6 @@ public class StoreQueueCursorOrderTest { } - @Override - public int getMessageCount() throws IOException { - return 0; - } - @Override public void resetBatching() { @@ -513,5 +508,10 @@ public class StoreQueueCursorOrderTest { batch.incrementAndGet(); } + @Override + public void recoverMessageStoreStatistics() throws IOException { + this.getMessageStoreStatistics().reset(); + } + } } diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/store/AbstractMessageStoreSizeStatTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/store/AbstractMessageStoreSizeStatTest.java new file mode 100644 index 0000000000..59ae44bcec --- /dev/null +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/store/AbstractMessageStoreSizeStatTest.java @@ -0,0 +1,266 @@ +/** + * 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.store; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.net.URI; +import java.util.Random; + +import javax.jms.BytesMessage; +import javax.jms.Connection; +import javax.jms.DeliveryMode; +import javax.jms.JMSException; +import javax.jms.MessageConsumer; +import javax.jms.MessageProducer; +import javax.jms.Queue; +import javax.jms.QueueSession; +import javax.jms.Session; +import javax.jms.Topic; +import javax.jms.TopicSession; + +import org.apache.activemq.ActiveMQConnectionFactory; +import org.apache.activemq.broker.BrokerService; +import org.apache.activemq.broker.TransportConnector; +import org.apache.activemq.broker.region.Destination; +import org.apache.activemq.command.ActiveMQDestination; +import org.apache.activemq.command.ActiveMQQueue; +import org.apache.activemq.command.ActiveMQTopic; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This test checks that KahaDB properly sets the new storeMessageSize statistic. + * + * AMQ-5748 + * + */ +public abstract class AbstractMessageStoreSizeStatTest { + protected static final Logger LOG = LoggerFactory + .getLogger(AbstractMessageStoreSizeStatTest.class); + + + protected BrokerService broker; + protected URI brokerConnectURI; + protected String defaultQueueName = "test.queue"; + protected static int messageSize = 1000; + + @Before + public void startBroker() throws Exception { + setUpBroker(true); + } + + protected void setUpBroker(boolean clearDataDir) throws Exception { + + broker = new BrokerService(); + this.initPersistence(broker); + //set up a transport + TransportConnector connector = broker + .addConnector(new TransportConnector()); + connector.setUri(new URI("tcp://0.0.0.0:0")); + connector.setName("tcp"); + + broker.start(); + broker.waitUntilStarted(); + brokerConnectURI = broker.getConnectorByName("tcp").getConnectUri(); + + } + + @After + public void stopBroker() throws Exception { + broker.stop(); + broker.waitUntilStopped(); + } + + protected abstract void initPersistence(BrokerService brokerService) throws IOException; + + @Test + public void testMessageSize() throws Exception { + Destination dest = publishTestMessages(200); + verifyStats(dest, 200, 200 * messageSize); + } + + @Test + public void testMessageSizeAfterConsumption() throws Exception { + + Destination dest = publishTestMessages(200); + verifyStats(dest, 200, 200 * messageSize); + + consumeTestMessages(); + Thread.sleep(3000); + verifyStats(dest, 0, 0); + } + + @Test + public void testMessageSizeDurable() throws Exception { + + Destination dest = publishTestMessagesDurable(); + + //verify the count and size + verifyStats(dest, 200, 200 * messageSize); + + } + + @Test + public void testMessageSizeAfterDestinationDeletion() throws Exception { + Destination dest = publishTestMessages(200); + verifyStats(dest, 200, 200 * messageSize); + + //check that the size is 0 after deletion + broker.removeDestination(dest.getActiveMQDestination()); + verifyStats(dest, 0, 0); + } + + protected void verifyStats(Destination dest, int count, long minimumSize) throws Exception { + MessageStore messageStore = dest.getMessageStore(); + MessageStoreStatistics storeStats = dest.getMessageStore().getMessageStoreStatistics(); + assertEquals(messageStore.getMessageCount(), count); + assertEquals(messageStore.getMessageCount(), + storeStats.getMessageCount().getCount()); + assertEquals(messageStore.getMessageSize(), + messageStore.getMessageStoreStatistics().getMessageSize().getTotalSize()); + if (count > 0) { + assertTrue(storeStats.getMessageSize().getTotalSize() > minimumSize); + } else { + assertEquals(storeStats.getMessageSize().getTotalSize(), 0); + } + } + + /** + * Generate random 1 megabyte messages + * @param session + * @return + * @throws JMSException + */ + protected BytesMessage createMessage(Session session) throws JMSException { + final BytesMessage message = session.createBytesMessage(); + final byte[] data = new byte[messageSize]; + final Random rng = new Random(); + rng.nextBytes(data); + message.writeBytes(data); + return message; + } + + + protected Destination publishTestMessages(int count) throws Exception { + return publishTestMessages(count, defaultQueueName); + } + + protected Destination publishTestMessages(int count, String queueName) throws Exception { + // create a new queue + final ActiveMQDestination activeMqQueue = new ActiveMQQueue( + queueName); + + Destination dest = broker.getDestination(activeMqQueue); + + // Start the connection + Connection connection = new ActiveMQConnectionFactory(brokerConnectURI) + .createConnection(); + connection.setClientID("clientId" + queueName); + connection.start(); + Session session = connection.createSession(false, + QueueSession.AUTO_ACKNOWLEDGE); + Queue queue = session.createQueue(queueName); + + try { + // publish a bunch of non-persistent messages to fill up the temp + // store + MessageProducer prod = session.createProducer(queue); + prod.setDeliveryMode(DeliveryMode.PERSISTENT); + for (int i = 0; i < count; i++) { + prod.send(createMessage(session)); + } + + } finally { + connection.stop(); + } + + return dest; + } + + protected Destination consumeTestMessages() throws Exception { + return consumeTestMessages(defaultQueueName); + } + + protected Destination consumeTestMessages(String queueName) throws Exception { + // create a new queue + final ActiveMQDestination activeMqQueue = new ActiveMQQueue( + queueName); + + Destination dest = broker.getDestination(activeMqQueue); + + // Start the connection + Connection connection = new ActiveMQConnectionFactory(brokerConnectURI) + .createConnection(); + connection.setClientID("clientId2" + queueName); + connection.start(); + Session session = connection.createSession(false, + QueueSession.AUTO_ACKNOWLEDGE); + Queue queue = session.createQueue(queueName); + + try { + MessageConsumer consumer = session.createConsumer(queue); + for (int i = 0; i < 200; i++) { + consumer.receive(); + } + + } finally { + connection.stop(); + } + + return dest; + } + + protected Destination publishTestMessagesDurable() throws Exception { + // create a new queue + final ActiveMQDestination activeMqTopic = new ActiveMQTopic( + "test.topic"); + + Destination dest = broker.getDestination(activeMqTopic); + + // Start the connection + Connection connection = new ActiveMQConnectionFactory(brokerConnectURI) + .createConnection(); + connection.setClientID("clientId"); + connection.start(); + Session session = connection.createSession(false, + TopicSession.AUTO_ACKNOWLEDGE); + Topic topic = session.createTopic("test.topic"); + session.createDurableSubscriber(topic, "sub1"); + + try { + // publish a bunch of non-persistent messages to fill up the temp + // store + MessageProducer prod = session.createProducer(topic); + prod.setDeliveryMode(DeliveryMode.PERSISTENT); + for (int i = 0; i < 200; i++) { + prod.send(createMessage(session)); + } + + } finally { + connection.stop(); + } + + return dest; + } + +} diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/store/AbstractMessageStoreSizeTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/store/AbstractMessageStoreSizeTest.java new file mode 100644 index 0000000000..923bc82c4d --- /dev/null +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/store/AbstractMessageStoreSizeTest.java @@ -0,0 +1,98 @@ +/** + * 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.store; + +import static org.junit.Assert.assertTrue; + +import java.util.Random; + +import org.apache.activemq.broker.ConnectionContext; +import org.apache.activemq.command.ActiveMQMessage; +import org.apache.activemq.command.ActiveMQQueue; +import org.apache.activemq.command.MessageId; +import org.apache.activemq.command.ProducerId; +import org.apache.activemq.util.ByteSequence; +import org.apache.activemq.util.IdGenerator; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * This test is for AMQ-5748 to verify that {@link MessageStore} implements correctly + * compute the size of the messages in the store. + * + */ +public abstract class AbstractMessageStoreSizeTest { + + protected static final IdGenerator id = new IdGenerator(); + protected ActiveMQQueue destination = new ActiveMQQueue("Test"); + protected ProducerId producerId = new ProducerId("1.1.1"); + protected static final int MESSAGE_COUNT = 20; + protected static String dataDirectory = "target/test-amq-5748/datadb"; + protected static int testMessageSize = 1000; + + @Before + public void init() throws Exception { + this.initStore(); + } + + @After + public void destroy() throws Exception { + this.destroyStore(); + } + + protected abstract void initStore() throws Exception; + + + protected abstract void destroyStore() throws Exception; + + + /** + * This method tests that the message size exists after writing a bunch of messages to the store. + * @throws Exception + */ + @Test + public void testMessageSize() throws Exception { + writeMessages(); + long messageSize = getMessageStore().getMessageSize(); + assertTrue(getMessageStore().getMessageCount() == 20); + assertTrue(messageSize > 20 * testMessageSize); + } + + + /** + * Write random byte messages to the store for testing. + * + * @throws Exception + */ + protected void writeMessages() throws Exception { + final ConnectionContext context = new ConnectionContext(); + + for (int i = 0; i < MESSAGE_COUNT; i++) { + ActiveMQMessage message = new ActiveMQMessage(); + final byte[] data = new byte[testMessageSize]; + final Random rng = new Random(); + rng.nextBytes(data); + message.setContent(new ByteSequence(data)); + message.setDestination(destination); + message.setMessageId(new MessageId(id.generateId() + ":1", i)); + getMessageStore().addMessage(context, message); + } + } + + protected abstract MessageStore getMessageStore(); +} diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/AbstractKahaDBMessageStoreSizeTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/AbstractKahaDBMessageStoreSizeTest.java new file mode 100644 index 0000000000..7d53cbd0f0 --- /dev/null +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/AbstractKahaDBMessageStoreSizeTest.java @@ -0,0 +1,147 @@ +/** + * 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.store.kahadb; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; + +import org.apache.activemq.store.AbstractMessageStoreSizeTest; +import org.apache.activemq.store.MessageStore; +import org.apache.activemq.store.PersistenceAdapter; +import org.apache.activemq.store.kahadb.disk.util.LocationMarshaller; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.filefilter.TrueFileFilter; +import org.apache.commons.io.filefilter.WildcardFileFilter; +import org.junit.Test; + +/** + * This test is for AMQ-5748 to verify that {@link MessageStore} implements correctly + * compute the size of the messages in the store. + * + * For KahaDB specifically, the size was not being stored in in the index ({@link LocationMarshaller}). LocationMarshaller + * has been updated to include an option to include the size in the serialized value. This way the message + * size will be persisted in the index and be available between broker restarts without needing to rebuild the index. + * Note that the KahaDB version has been incremented from 5 to 6 because the index will need to be rebuild when a version + * 5 index is detected since it will be detected as corrupt. + * + */ +public abstract class AbstractKahaDBMessageStoreSizeTest extends AbstractMessageStoreSizeTest { + + MessageStore messageStore; + PersistenceAdapter store; + + @Override + public void initStore() throws Exception { + createStore(true, dataDirectory); + } + + abstract protected void createStore(boolean deleteAllMessages, String directory) throws Exception; + + abstract protected String getVersion5Dir(); + + @Override + public void destroyStore() throws Exception { + if (store != null) { + store.stop(); + } + } + + + /** + * This method tests that the message sizes exist for all messages that exist after messages are recovered + * off of disk. + * + * @throws Exception + */ + @Test + public void testMessageSizeStoreRecovery() throws Exception { + writeMessages(); + store.stop(); + + createStore(false, dataDirectory); + writeMessages(); + long messageSize = messageStore.getMessageSize(); + assertEquals(40, messageStore.getMessageCount()); + assertTrue(messageSize > 40 * testMessageSize); + } + + /** + * This method tests that a version 5 store with an old index still works but returns 0 for messgage sizes. + * + * @throws Exception + */ + @Test + public void testMessageSizeStoreRecoveryVersion5() throws Exception { + store.stop(); + + //Copy over an existing version 5 store with messages + File dataDir = new File(dataDirectory); + if (dataDir.exists()) + FileUtils.deleteDirectory(new File(dataDirectory)); + FileUtils.copyDirectory(new File(getVersion5Dir()), + dataDir); + + //reload store + createStore(false, dataDirectory); + + //make sure size is 0 + long messageSize = messageStore.getMessageSize(); + assertTrue(messageStore.getMessageCount() == 20); + assertTrue(messageSize == 0); + + + } + + /** + * This method tests that a version 5 store with existing messages will correctly be recovered and converted + * to version 6. After index deletion, the index will be rebuilt and will include message sizes. + * + * @throws Exception + */ + @Test + public void testMessageSizeStoreRecoveryVersion5RebuildIndex() throws Exception { + store.stop(); + + //Copy over an existing version 5 store with messages + File dataDir = new File(dataDirectory); + if (dataDir.exists()) + FileUtils.deleteDirectory(new File(dataDirectory)); + FileUtils.copyDirectory(new File(getVersion5Dir()), + dataDir); + for (File index : FileUtils.listFiles(new File(dataDirectory), new WildcardFileFilter("*.data"), TrueFileFilter.INSTANCE)) { + FileUtils.deleteQuietly(index); + } + + //append more messages...at this point the index should be rebuilt + createStore(false, dataDirectory); + writeMessages(); + + //after writing new messages to the existing store, make sure the index is rebuilt and size is correct + long messageSize = messageStore.getMessageSize(); + assertTrue(messageStore.getMessageCount() == 40); + assertTrue(messageSize > 40 * testMessageSize); + + } + + @Override + protected MessageStore getMessageStore() { + return messageStore; + } + +} diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/KahaDBMessageStoreSizeStatTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/KahaDBMessageStoreSizeStatTest.java new file mode 100644 index 0000000000..bb46f20e55 --- /dev/null +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/KahaDBMessageStoreSizeStatTest.java @@ -0,0 +1,82 @@ +/** + * 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.store.kahadb; + +import java.io.File; +import java.io.IOException; + +import org.apache.activemq.broker.BrokerService; +import org.apache.activemq.broker.region.Destination; +import org.apache.activemq.store.AbstractMessageStoreSizeStatTest; +import org.apache.commons.io.FileUtils; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This test checks that KahaDB properly sets the new storeMessageSize + * statistic. + * + * AMQ-5748 + * + */ +public class KahaDBMessageStoreSizeStatTest extends + AbstractMessageStoreSizeStatTest { + protected static final Logger LOG = LoggerFactory + .getLogger(KahaDBMessageStoreSizeStatTest.class); + + File dataFileDir = new File("target/test-amq-5748/stat-datadb"); + + @Override + protected void setUpBroker(boolean clearDataDir) throws Exception { + if (clearDataDir && dataFileDir.exists()) + FileUtils.cleanDirectory(dataFileDir); + super.setUpBroker(clearDataDir); + } + + @Override + protected void initPersistence(BrokerService brokerService) + throws IOException { + broker.setPersistent(true); + broker.setDataDirectoryFile(dataFileDir); + } + + /** + * Test that the the counter restores size and works after restart and more + * messages are published + * + * @throws Exception + */ + @Test + public void testMessageSizeAfterRestartAndPublish() throws Exception { + + Destination dest = publishTestMessages(200); + + // verify the count and size + verifyStats(dest, 200, 200 * messageSize); + + // stop, restart broker and publish more messages + stopBroker(); + this.setUpBroker(false); + dest = publishTestMessages(200); + + // verify the count and size + verifyStats(dest, 400, 400 * messageSize); + + } + +} diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/KahaDBMessageStoreSizeTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/KahaDBMessageStoreSizeTest.java new file mode 100644 index 0000000000..43dc2f6f2f --- /dev/null +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/KahaDBMessageStoreSizeTest.java @@ -0,0 +1,46 @@ +/** + * 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.store.kahadb; + +import java.io.File; + +import org.apache.activemq.store.MessageStore; + +/** + * This test is for AMQ-5748 to verify that {@link MessageStore} implements correctly + * compute the size of the messages in the KahaDB Store. + * + */ +public class KahaDBMessageStoreSizeTest extends AbstractKahaDBMessageStoreSizeTest { + + @Override + protected void createStore(boolean deleteAllMessages, String directory) throws Exception { + KahaDBStore kahaDBStore = new KahaDBStore(); + store = kahaDBStore; + kahaDBStore.setJournalMaxFileLength(1024 * 512); + kahaDBStore.setDeleteAllMessages(deleteAllMessages); + kahaDBStore.setDirectory(new File(directory)); + kahaDBStore.start(); + messageStore = store.createQueueMessageStore(destination); + messageStore.start(); + } + + @Override + protected String getVersion5Dir() { + return "src/test/resources/org/apache/activemq/store/kahadb/MessageStoreTest/version5"; + } +} diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/MultiKahaDBMessageStoreSizeStatTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/MultiKahaDBMessageStoreSizeStatTest.java new file mode 100644 index 0000000000..4342e1d529 --- /dev/null +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/MultiKahaDBMessageStoreSizeStatTest.java @@ -0,0 +1,134 @@ +/** + * 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.store.kahadb; + +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.activemq.broker.BrokerService; +import org.apache.activemq.broker.region.Destination; +import org.apache.activemq.store.AbstractMessageStoreSizeStatTest; +import org.apache.commons.io.FileUtils; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This test checks that KahaDB properly sets the new storeMessageSize + * statistic. + * + * AMQ-5748 + * + */ +public class MultiKahaDBMessageStoreSizeStatTest extends + AbstractMessageStoreSizeStatTest { + protected static final Logger LOG = LoggerFactory + .getLogger(MultiKahaDBMessageStoreSizeStatTest.class); + + File dataFileDir = new File("target/test-amq-5748/stat-datadb"); + + @Override + protected void setUpBroker(boolean clearDataDir) throws Exception { + if (clearDataDir && dataFileDir.exists()) + FileUtils.cleanDirectory(dataFileDir); + super.setUpBroker(clearDataDir); + } + + @Override + protected void initPersistence(BrokerService brokerService) + throws IOException { + broker.setPersistent(true); + + //setup multi-kaha adapter + MultiKahaDBPersistenceAdapter persistenceAdapter = new MultiKahaDBPersistenceAdapter(); + persistenceAdapter.setDirectory(dataFileDir); + + KahaDBPersistenceAdapter kahaStore = new KahaDBPersistenceAdapter(); + kahaStore.setJournalMaxFileLength(1024 * 512); + + //set up a store per destination + FilteredKahaDBPersistenceAdapter filtered = new FilteredKahaDBPersistenceAdapter(); + filtered.setPersistenceAdapter(kahaStore); + filtered.setPerDestination(true); + List stores = new ArrayList<>(); + stores.add(filtered); + + persistenceAdapter.setFilteredPersistenceAdapters(stores); + broker.setPersistenceAdapter(persistenceAdapter); + } + + /** + * Test that the the counter restores size and works after restart and more + * messages are published + * + * @throws Exception + */ + @Test + public void testMessageSizeAfterRestartAndPublish() throws Exception { + + Destination dest = publishTestMessages(200); + + // verify the count and size + verifyStats(dest, 200, 200 * messageSize); + + // stop, restart broker and publish more messages + stopBroker(); + this.setUpBroker(false); + dest = publishTestMessages(200); + + // verify the count and size + verifyStats(dest, 400, 400 * messageSize); + + } + + @Test + public void testMessageSizeAfterRestartAndPublishMultiQueue() throws Exception { + + Destination dest = publishTestMessages(200); + + // verify the count and size + verifyStats(dest, 200, 200 * messageSize); + assertTrue(broker.getPersistenceAdapter().size() > 200 * messageSize); + + Destination dest2 = publishTestMessages(200, "test.queue2"); + + // verify the count and size + verifyStats(dest2, 200, 200 * messageSize); + assertTrue(broker.getPersistenceAdapter().size() > 400 * messageSize); + + // stop, restart broker and publish more messages + stopBroker(); + this.setUpBroker(false); + dest = publishTestMessages(200); + dest2 = publishTestMessages(200, "test.queue2"); + + // verify the count and size after publishing messages + verifyStats(dest, 400, 400 * messageSize); + verifyStats(dest2, 400, 400 * messageSize); + + System.out.println(broker.getPersistenceAdapter().size()); + assertTrue(broker.getPersistenceAdapter().size() > 800 * messageSize); + assertTrue(broker.getPersistenceAdapter().size() >= + (dest.getMessageStore().getMessageSize() + dest2.getMessageStore().getMessageSize())); + + } + +} diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/MultiKahaDBMessageStoreSizeTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/MultiKahaDBMessageStoreSizeTest.java new file mode 100644 index 0000000000..398b2f730b --- /dev/null +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/store/kahadb/MultiKahaDBMessageStoreSizeTest.java @@ -0,0 +1,68 @@ +/** + * 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.store.kahadb; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; + +import org.apache.activemq.store.MessageStore; +import org.apache.commons.io.FileUtils; + +/** + * This test is for AMQ-5748 to verify that {@link MessageStore} implements correctly + * compute the size of the messages in the store. + * + * + */ +public class MultiKahaDBMessageStoreSizeTest extends AbstractKahaDBMessageStoreSizeTest { + + + @Override + protected void createStore(boolean deleteAllMessages, String directory) throws Exception { + MultiKahaDBPersistenceAdapter multiStore = new MultiKahaDBPersistenceAdapter(); + + store = multiStore; + File fileDir = new File(directory); + + if (deleteAllMessages && fileDir.exists()) { + FileUtils.cleanDirectory(new File(directory)); + } + + KahaDBPersistenceAdapter localStore = new KahaDBPersistenceAdapter(); + localStore.setJournalMaxFileLength(1024 * 512); + localStore.setDirectory(new File(directory)); + + FilteredKahaDBPersistenceAdapter filtered = new FilteredKahaDBPersistenceAdapter(); + filtered.setPersistenceAdapter(localStore); + filtered.setPerDestination(true); + List stores = new ArrayList<>(); + stores.add(filtered); + + multiStore.setFilteredPersistenceAdapters(stores); + multiStore.setDirectory(fileDir); + multiStore.start(); + messageStore = store.createQueueMessageStore(destination); + messageStore.start(); + } + + @Override + protected String getVersion5Dir() { + return "src/test/resources/org/apache/activemq/store/kahadb/MultiKahaMessageStoreTest/version5"; + } + +} diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/store/memory/MemoryMessageStoreSizeStatTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/store/memory/MemoryMessageStoreSizeStatTest.java new file mode 100644 index 0000000000..755936ca69 --- /dev/null +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/store/memory/MemoryMessageStoreSizeStatTest.java @@ -0,0 +1,45 @@ +/** + * 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.store.memory; + +import java.io.IOException; + +import org.apache.activemq.broker.BrokerService; +import org.apache.activemq.store.AbstractMessageStoreSizeStatTest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This test checks that KahaDB properly sets the new storeMessageSize statistic. + * + * AMQ-5748 + * + */ +public class MemoryMessageStoreSizeStatTest extends AbstractMessageStoreSizeStatTest { + protected static final Logger LOG = LoggerFactory + .getLogger(MemoryMessageStoreSizeStatTest.class); + + @Override + protected void initPersistence(BrokerService brokerService) throws IOException { + broker.setPersistent(false); + broker.setPersistenceAdapter(new MemoryPersistenceAdapter()); + } + + + + +} diff --git a/activemq-unit-tests/src/test/java/org/apache/activemq/store/memory/MemoryMessageStoreSizeTest.java b/activemq-unit-tests/src/test/java/org/apache/activemq/store/memory/MemoryMessageStoreSizeTest.java new file mode 100644 index 0000000000..19a01ab77d --- /dev/null +++ b/activemq-unit-tests/src/test/java/org/apache/activemq/store/memory/MemoryMessageStoreSizeTest.java @@ -0,0 +1,45 @@ +/** + * 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.store.memory; + +import org.apache.activemq.store.AbstractMessageStoreSizeTest; +import org.apache.activemq.store.MessageStore; + +public class MemoryMessageStoreSizeTest extends AbstractMessageStoreSizeTest { + + MemoryMessageStore messageStore; + + @Override + public void initStore() throws Exception { + messageStore = new MemoryMessageStore(destination); + messageStore.start(); + } + + + @Override + public void destroyStore() throws Exception { + if (messageStore != null) { + messageStore.stop(); + } + } + + @Override + protected MessageStore getMessageStore() { + return messageStore; + } + +} diff --git a/activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MessageStoreTest/version5/db-1.log b/activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MessageStoreTest/version5/db-1.log new file mode 100644 index 0000000000000000000000000000000000000000..18f34cf19da59063c3d4fb309eb3854a64175f12 GIT binary patch literal 524288 zcmcHAbC9LUzAyZ$E?Zr;ZQHhO8(r)&yKLLGZQHhObouR>JG0NdXYV`jJuxwBMa7d( z=2{u?{Ubk>^;G@<006MQs49pn3F8UyD+!7L002<^`T=P40Dt^6NW5 z$B&3{%3ia(8P$@R1mjlpO`g{ozAlprrJ?l0&$WZKtQ}OV z@}(dTnIKBEvk8PiYDJRTQ?|CK7!>bJ0YEzMOag5md43e&YqWnF#;{Ed!wu*Kny5xR zxv275dVV%(D*vV3iw@;G0ZW)38W6<1{W~Qs73D(5^33XxD5-4Cn@9e$)RKK6a;RD= zqls++&Ju(*h;@wqq^n_w6~n{|iHfFq&z_?eyjTyg;#3%bZdWa=1){Up3nO?`mm)T1 z3loO`m+XL{@HwAjjU;yB_cX6hBz<)_@U6kYty9fOE&k(YA!uT;9@=k%1|@8Y!SKNf z8CRjmo+EF_79Wh?q!OEVi)P43AQ~58c%_S0691%Q7_vGKH-t6|gXB2jRnL znV2YmuoacuMP?2*VEOW<(grolfHr8&BiO~MqgKO&sM^Fph!u z02hETGR*6%+1XU3Dr~~${pwfeWY&Zf|Dnp5B-Ss-y`tzd7zL62`>8jk`c9E4H0u4dee+J*8j^)?p?9ccNuF2%)!jZk0l8nb*FAtb9u zV@`m`LdTQY6qdR5YoaCykxExZ9-IS?rS9w{M9rW|En3 zKe#lS^gXx85DYxXE0FZRT)%}AVF~%@F}8nbr3}p+u)tOeOKy5Y<7@9$el*slF@%un zG-iQaU#B!03gLPYOM6|sl`g=8C(RF7k`K`LLS4Fd zl9T+o2%WjOL|6sFjDgqqmlFIrNCrUnTk}8icY6tu|F`*L_zl2+kiRc~B>9uWKZE_J z>+k$2zyz83C!rktSV))hKzO274B)E!E>kbiU0gRez9FxD=r`Y{hM=Jo3Kj-rY}_vY z)10Yu96?T=*}{qXRJ$avi!3z|wr+jKI?)M-kvYMF5jF-+24+DNX5kLFdh`20ae zy}@14@~)IIjybUtM4&1L7RExcd^jb0S3ECe;y#lq2ANNSWXMeVq-9wj3PV4KAT+r%-MU3LAr3eGe3 zCI(_DJ!D$&u^ym?2UWE}PBc%bZY^&e#2gM=xZ`1yYf_zoaeJO2)dsJKXhFs#DesFj zg5y#yO$}tzN$DFK>{e`FgFfiXH9u!WmmRk~M3Eyye=NvlUvL$IL=_-y0z^^J36%`P z^*y4*QdQD&h?m5m+2@c~$nJFp>gyEA?xNXcrYn?0Tal7-@6>Ebtl$FUNQZm#m^+6UB~5&Mw`(UY*=xhB=AOZdhO@T^JxhT#xFv<3Genl!M-EY zT2oc_ugg({b_zVhMrDjPe!bLMTOhaXrli;uj$2>UL0|>!Xs}Ujs9bMdxV5jPHn`iT z=b;85!wZZ0^xQR+s3XrPezF`%razU%Yt_G1eveP&k0T_%@z_$dRCL7nwELz?QyxiC z^yr)gd$3Ff$&?YgJ70heM1jV_8<2;~@3=&|yRpX+oBn#f3RWDVb6_sL0xZkR&b)3% zcZHKJigcB!i`OCqT5|57oakQ&k7valyDg}Jpan+nHQkXNJ(OQQPDz+k#Ys6K#^yit z$hgzD)|LXbbxyEOD6qjy>jP_8CFk=czse1`YcuJqIm zXVy|fiBq8BlLcWtAx}2FJNjvWX)`KE7Iu1D+nQJ`f1m8BQDq);q1@fNeDDrCrEfHY zi5qO%(`TMCj&34IMn5C4Motgo~`T3x$K;SbQ5$iBZ&Vk?~G zuVJgLK?@mRpgcgX_|*aXi|Ck_0 zfXP=1H+{!3JmwHMNEO)gXYiqhK+CzuOU@QhOWm|y6{)tUoeEpUABckTMTYG#1&F^H?H)T>|rf?dd6ldhnFR_u$I1)d;n^)0z)=WHUWcc!z(*am|h4_WeXcYDbbNps(7U zQ9obTQ{N(nMC!!pe`{(jU`~~6;0TKY>>I4BQhCj}SxMGUay{4{_qE;BfdfLkIJv5I zuwhL9p}5C$McEk3$zg+=Lv?{+di)&+f=StFPAe!sN@Tf;qJqn&oT+Ry8_nrNkV~-= z9w)El8p{v7K)&e2a^re(Xf8h|g@Br>T=m}O!HcF2AqT18DkdJ3IfL~VNs5dQ0C?13}X&odVg%OF!O$fj1&rS z5@(6do*Iil_n{9wZ?P!v)h{&z30&J%I7vu{kSHy=Sg(WCR*^(5*G*W{HYN2v8=k(P z)=i*jSRspcAP{keWq5K{JvBWze!U4+U$~_<2o@>f;Xr*tb^u)ai#fM*N`?+_@#u)) z5c4+Sbjv!glWGTcE^ve*7xezy+W*hDf5o3ID&q9t=8x$Ye~kYif53m_`jf%`9P@wm z{?6Zr1vz$}!sw6))OS6C_=GE2<}ON#tPz@$=or*+Ife*aj%ds)-bM={q;jd7bRWV= zkUFq%aH}C0F^^c=URk<_eegBv%HiQ8e6o?+z2;|H+Qx$)*kU=3 zLrnt9X4hl}Pj(S)eIqcPNTS)MKJvcxqEWDLQeur+4;qI^8W70a)}-I;gua*=+t7uf zF+{GohIEik@)0pjB=(exd()2IL><;-4Ls=j^5;#9c8~x~VnOt>@e+jY^t4f45mz?6 zXC)Kqrj87PtSx(eg%wWriL~&?Rwya`f(a&Xvl6W&>y|dO1ZqZaO>z-9)6!q$LF4F> zb&H{n-5e8>Q(=ynEQW11G}!^lsE+9vU=L)ZJbfeG^f7v)gB*yYpV%m{qIHruM@Z&H5suw7Wpmobq$P zAi6jH`*x1q<0`qgQH_mlW8@?4nsB2Z8uR*&kp@^?tzUAK_LbWaH?ne9pQeiOK^FR|czpb=;OG6f6X zzWhVskVcepw1 zJH@&RvsO4#;RTgM8@LvyOSbm|pCA=>fW{rVmi66%I$Af0IlcsY-84P8!6j!_Z%UaZ zfpXtZ_*Au@ZGg*^1B}g#BCFp+l3-QFd=>ol2q_#s=D9K%Wvp|zp{h})dz16a8@Lf8 zb6I%K{8mborL5-KKs>1fkMARYn_*bAmBSMEv@RS6%+C9;KUNSI)pq0Cl(Z;-(m}gR z=k{|s6DE-ks%G$v7qrlDegw9z7OL9&!Bpshn$B&Pz5Zn0!W)?%Yt2z>)NGklbx5=} zDFwd=zZ-U#)qTwT+Eaiqx~BlN8$@|UMK6J~s;e3K@QnI_!G^oX)6&lM705L{;V|f8 zl@D;+$JiBgjvC6Y!YF2yGXG&d*)rp5UM~$Zu{4L&ILX;oMbB#>Gv~}`kBaOK8-Cuh z+$A%7NbB7=E6==9pD*0>`dSI<)KGuLN6V54TN#29blq%FwKS%l@K>Z9Gl&>LPF6mB z^ZF>zvyLdLS^jX0hJ&0i3))zPWbm8_@zM}^LFH|e7X`$~R9cp)j#ms_se|KYk;Aa9 zjI?0~6=AcSo@BUvCV&V)7=4TH5Co5U`<#?$v2qako3Je;WHgwbStC;{h>ftk4Moiw zkd4d6N$`@3-!Q8DznrZye*P5%{~G%%{tk06-2XOz%)j_!`Um*~`6J4o4E`(Ge`x&9 zUm!^nVHsW-hEBC*!7Dkfbu0@lN2dJROl%+~b*lA83$Rt8e=;u(f%}UvmiJ@nMx4;F zWmF&(EvHKJ(110-mhNOyvma`%m@yg&#GJOzG(z^w&;)-MbRTI*#tb8*7jIRqZs&-w zlwwH~pQ@3k8*<56sVPwAZd{)}qG=H{8ePq1OT%^b@yE=`afbK(&O~X~cuz;f*}}MW zQ@&3l0f8M%hzZS$9>K}A%sDtt093k`KjI8w)^mM#TZ(~2&B@4Y=niC!Iv^9Ee5beY z!H_Md-jUp7E<-nO<>5VlC{`&PMg5v+5j*1ak$zgpo6MclUF>CWA~kZ( zOr=Hja6SsyP?YKk`lVXHfRlr}{N;2A;N~M0lMW#}w5syAe7u0opV|FWk}xt(*An8s z13^#9P#oRZSmTW3sF(`l>v)tv=VhZg?3np_9z9e;OY0J#xwoqDrolnQR68rFvad?Q z!QyGFMcLTdI~43?B3(s{RWtF%2gB5BX>kxtJ_anww$=+bCsN#9pGV~0ToNhL+!t6o zwaxrir+yo_((cxe+qNvsMJ6)0{#xj$>e5Y$(SS`4g=%P+G$Po(_c~m$++lv(v!aHi zT889Iss@h4b-HC#+AdV}lmu^TMLnN5X4lWlpPHOj~c5d3G*o{Rkf=>kJxQLW+q zjFLH`blabngSS=7d?1}Pg$dLPji2Y(op{K%LX@5V;1Jw7|xP0AtiuM?}Ut)(4N2iacbJ`S1T{vT(S!4^9`f?aA1woD$&1nu0d&V}X zFIfG?=7P5@tWMA#W&XqZ|M#xH;;$+Vf9-Gc$MTCm=6{es&_6Q$$>6_7{I8zh`6I2X zE9)M%ouO)4w8${!-Y`bqFdgz_X!Zo!N=GJq6+zJs#ghMGY0O5+9QS;8H$PcpT}eh7 z_e5Sq&<40I8L~#z;wOJ_R&8zaHBhvU@1V`TN6p$hd*ySZt3x_fZ*y`*ipgN13N-e8 zx=v^ruuF>c-GgH?A21uc_b?1Q#KQHi++E^LZnT%BBbu$a5p^iLb+1ZI|556DmI?JzVo` z(9ewXXLg}$x6M;hYwkNE!)QL~Z`IR%n7o~q7*bkjWKIM%dM4MPw>bJ9r@n&wpP!;v z50V~no}JD(I)JuR;DOu$$SaRUaAt6RVB^7{IvowlHB5fQM8o<5I%o|uFzHHZoJ7nN zNE1y|MY4fJ4n<=C?`AP!V<9D9u)`3Up8<)Kcj4t8GCI2;{#Q+ z@uWFPz9|x~$kgqTL5nL7K_Qm3POy>QT2#5wZ=ukNm%YaI@gAae%jK>pkH6{J+4XhH zz47g(B+=wa&o0{sNV)!SF>)EMUz`db@Q%*3KTRx*t=cIZf!CB>xEpF_AFY$8WZm(Q z*mU34OFNW;BU6vuvgM>cALKe(9*hFaQqq`dPLN=cy=oN;|ytGruX%x~D0}4lZ$0U2GE+c2^Hf zx_EIQLd80x(v^MuIQ`SiHPr8`I3NPQoj{f7uv;5T)q?DpSn!AJbvMy*w_g)h(~oEJ z5oTXQuV!e1@6dB4E4Lr~-?kmh)Dl^>!um90vLP&DuG^*ttO zI3N?8E0z2to~dRPvT@GhTa*y+`id*{A!38Ln^E> zc+?t4r6S6Nk-YI!8wZvzwT*0!ml)QBkAp3Stg5KRxph(t@+Xy$_5ej%et0>*WS%+b zq{ol4rhxRZJa+Gc|62Bc(fTX?NSG-*|2BWDzxZSM2l)g0Bix@1{zcZ`{#y)mY?ALy zV0iEH-kJpSUke#RM0VuSet&<*9(j6LkAi( zLiOP-fLHAi|0-u6a&vTP&E)<)$uH1V3^}I!EYzonX8~cDCZ58MGLf#cek-6iVo5lB zd&qAScW{#K-Sv1r?@Mq%fXV^=lCCfhZZ+R`N!;}=gOWOIV*7x5q%eq#cmCy_(Sil)V ztD_kLHcPv`e;?KEo0opZ?GvtdFKpiO>8jSb(I>_(%8a%1jaD=*CEPFpkRT>1+jI z@T}B5dIbBCUoD-*j>=H)kRV!|Rr_9~5QSsX?gBbxXrgo96Xi?lyZ!Qa+osaj~ z126?9TTM+CaSpVdLTzKY!^UQwhS^C;rU^VSv&qTvD$dKpDn`B~-0o*2aW)pW(Q({0 zlR@hSK4j$zZKhHF4GRNjVi2e#Gj7lU;uY32QMIbunLeLS1Ch00UE=rF0GXCnXbA~C zK4;gwO%*rTR(}bm4(L8*t}fX!reH7#N(NyD2k+I&;hi-IakUQ`Ct7<+yeF%x9Kn#Up|5??X8PBKze>gD*8}DW=6s}kMRkY;H@_H$x4WQ zPtQ%h9CDAk(Xzn5g5Y0cf5qS3H^|n%%^%w@{#gG({=ol;@+X7;O7Nu89E=ORSD(CLU+Lv<-|D7!4Ie#QUwLx>^EoT9PA5NcZYUZE9 zWAJ>43bKpoh?DaS&yCMc10oqkwf(}53Yl=1>i99DthrR0BPfMfPZu9AtP_nCdrMG2 z4ILN(;nUS_7)}u@uOb}l$@$r>6bBZUy8X0M)CvGfQO>Mz_4j|uD>|}|% z_Z0^0G8vmJ2}qR(b&vXJlL`zn8AN08CCecsH_Hdb6lOR{W!`c-*lxnx}^sc;TO0KDTBL!$z$e~*wncRK4rJ}R}nr~OK%*8 z1QxOE?6DzWX~Su6g|B(AYlkTzCuw{4Ym+`k#%@Ne0T|WOQXJ2Fz-uBN)Q!CD@vgsq zy+}vl%pYF1(=dPCVd7ipM+@v(aM>H8WM3b0S8Q^aQyzAJkXUvx*-}=aPcI|IZx%mB zza2+`&~>F?w6(#PCXd^1;;Q3c<(8qHrWIw+9NO1w3VRj7r|v(Mo%q%vmOYUS%3bgl zf%86kl@<&j>0X`_s7LR(Sf00+3C{`wC@4k9k8 z@7~GIad^*?GSufk`UYB&V>sfH0q zZ!c?vl3H)9)qTW5N4Rup-v-3(9%6mc^*H#!xs~hIL{vehz+5)mbl>>oEql`TOKD8y z22dlLwXMv74GB=#Vly?_pznsqrEkuQnna-^ML5S*OBlNFySgK6EAoR8mAb6t#GWC9+H&gKs~m%Kft4tDhI5f z5Re7sgg~ad-sgPRkii6l2m#Z+z5+8z6`IscWD*e~jne1$5-YMgAlon}y_qC!DF~VJ zDb=0w0y{An(RqUFOIgx~ouWT@vZHj1i{ypjwsT^tEKjVt;b#_2fHTR_CafZNkYGQVM)Uw#L+;uSkJ#V)3n(s-syY)#p{ zC#*n3oeWtNM*!^0^Bcuc;{WV|{as6fm#GZ~lOanA3G~-XL!4 z)GjcOVGVG&B_psd}bpoeU-58mKM3A-m~aEl5mqG!xc7fUL4pl9Fry!R0nE z`KwThQr>25$Xnq{bcTGpgX5T?`U=^?$K9lcWcVE+%J7ETYQjB9K1Qvf87j&H0@8hS zJSZK|39bj9h7br{Cl~5x-`1gz1jYY}exm-h2$eM*8)pUiil)yrS9P$SnaT z0**};A;DQ=jl`N%V0`ZDc2^cZbK4Y(Y?Fwe(xmX!Ky9wSNK3*k!4fXoDSI3zI!ROB z4Zk(N(3hGy=tVW$mPY>U9A@2T851ejiyr?>;?JD+j?L^%*SMKJig9|<^foSaR@P_U zdzYxF%H~v9zbWHmX_%tB)J(=YPI@@CXl)wN$DZb zV&>3OXff|+^3=tR(_`O$2)QeSGPRXM&UfX=-<<>Opd%P`$OKVLMDe;h8LnhIPEQtf zqtqZyt*)}^6Hs=AbL<(<17|OBy>uu7*Fx{5c0ePxUb29f^0@;#0YKz3)FhQG*TWipG9W@5|lKC|?YlMEJdudUPzzpNX(5?>*ehwa&3 zJ)v}$bwg$)PcS1Zs1x5z)je6$?oa5dD#hG87(xgV2DN*@Eq787(-;s^5Hv1?!5jBR z!rBpb8u?cc{A=w0!r%UHAE>%O4*A93J|N)w|9S6k_La;PbtXm24@kX{AUQYLtE8=p$1nStmJ~(_w&2DEYfT1-%T(2}hX9}A zR06{;0R%P_jNF?&eu1sF&%`lZJsSsV55x0D<~(wF)K>YJL*t*(>(nNCO6s9Nn&Z_8 zB`+qPJO@S#5gOS4jr<<(WYP-S6RL--1Q$#eQvExU`9Ew%nVg_2S}7kA?>h#5~0t|kdo^ae0Ls0Ad6i` zZLJmLydQ@DWEY6zjbI1%7Y zMqQU*c=Jo->8;#B$Es<5j*?S1+G)A}zO1ZD1IfFa9}N)ogMg(`E>i<-x--0ypH=fx z%{{@(;p!Z_#(^Z$(FnBIq#2EUBv1Q#2oLf5bT^J5x{0TwBy+HYygGsW1K_=2)dUZV zOJS|*y$Zo+oz=kDD7FK%Q8pNIvJ`Eubeh1*?qSV-+6JtQ6Y$)(e4lxU$BFTr!jU0v zlF~XKvyxlZa1Y9v2onh<@%{7Aj{$JSrB!^7O((t*Sh8QB@I%|K$OsMG+`++;?i)6Lv6taCNv=!!D6 z(+c%Vlyxwq?F`aYv+37Q#keHiX4Tu*mL9~XmoEcH7oMm5)JP2$3W-77iUD{_h#oCk zW0lpQfTO3~FsWM0RM%5E$-uO=oBv_`|9jW}#lPR59IB2W4FB@)01$BFfA^2+f9BtB zbszNqWZ?(YAF2L?@ZUrJSI_SNx^D}0JK88lNvJq%yCDa)=jDr_7mF>1H&f~ZwBu|w z7581&^t~OxaI>9AfU#9QOKoS>q2l7sNM|!7Yv(e25eq9>hq(9ChyD0)O!nu% zL3~n|FPzK&%I=fICIhVdWY3~JF~un~V+H-HO^obvdxVj<=5~CMZbasJ5qA`xC93%% zU&r<}xGFw1&^th$ITX%&=16xnr%=ZZovdVgcqd_eN%tr;FExW(Oc?eo4~7|@4N8dR zTw0bC$i@>h{}Rsq#V5e)Bp>Vi*uBf zszAW8WI1b|e8YKg5idZfpf~9;$SY&L@mbR#rfs$f^~o3mE#k%+(?EQ|$I()Ah<8)h zLo4cmHbHZ&H{la6{aVAB-4$Q7U=Dc|Y^65RLQ+++jHNr|d|6+o$}$pq*2NsKmq#xW zZ9`SFVZqqeT?X4QEY9L>(%>%DFqo&|B+^$00AZ!NQ&yeB067Gpx#5TLI=48?r4}}e z&7B!NyyNV>ykf)%qx3pkRMJQp3MT7BP1e}89+_?H!r^k?C31zbMRf`#AN*8YLbUW% zYXboK$o$*OFLKL;a5Ku2ARHA{r#rBmM}q0VLO`diWetvG^n}bjDhL3S1mePdX<1H@ z^B90$6{6&VDcps=Vh>Q?42p|i!jA~i$*7$r^8+U2N*WGC^Xg7GBVgT9W2dQHx_HSg zSjy0SSgV734_FQr4(ct>4Ce@4Nf~8vij>kWPW@@#9@s9l2wLjNz7_NFyIuR0t})ZW zI%(a{M!4|jzK~M*iQQuPF&Kp3GG2K~$Hx-{K~R6#HdlG(2u1x+p0{Tyz~nk~=A#cpzxq#2efx0>`098kaF{ zwlC=Afv#jmC2ny}>t)*F6r_Ew;e~FdZpa!#axiDK@3wpL1$^zI6p7b{w0)zqF>=v3 zkt@!(fJmzwG6`4xAE7Uz5-u<-;3ntf^{!^xC|GV6gWzvY$`m0uP6W#Rsql@rRKT)S zc`7g!iCZRbc12>BHF4W14>Mmv$wX+1GoFPbhdQN2Mkqhjy@AN{Sre}4z?M34Me{oLOL@b}7( zf3oxg`VZg!gz&$(`}fxG04kzDH?r9W2}^PGpGk7RxQP}HN`Em}$c4>mOSUQ?-hmt> zlSJz5Cw4}qAHFdaBM0!ti5s>w<0f2d$&R7UD6E@rsauzb&-Y;EBGUA+&)j2=$T$T7 zSlLs+YFyk6Jm2Y!x}y7uT1kN^g>2rytc_KoZ;(JaO(_flVfP+kt^an1)p=f@A#CbO^iKCUhI|5Mh@CR-#)w?a5^t(mWvohFkI`lo=JQhv8~5{`?t=h=ZsSH@YHz zlG*Zkb-5F1hL0B)@l(Dt1dsfd4#rQAALpx|2=?nZ87Y-P)%gWbx3KNXRb)oI*lPlT zt3j0m(4zxjNJ3e+T2AQXQcU0!IR5NB{!!i=r%fWG@uz3_^~FFjh^|Z_od(;^cb|{DhB!xFa7%$6Ufp_wFhU_Udfp0sxj5tl~c+bkRCFA1rDdp!`D#UPm;v!O>)u4tM9h_Nnbyo2^KVWtn7p5 ze!lA`EeV}Q_!7E8pPugFPW#3XhyyN3;(6BQIvtu>ruZ&C*+8Lcm)^by(wK5~R%lF- zNaq~Cg_Z$#ZW~l0SF>7Ss~l8+E@wP^mn6%8d|*73mVjxhKgw=W5nN!LrN{6TqBCjGA`hnE-_DFp+ zgj72r=`74VYtuTyCGu^7ljjWxUH7P6$BO-Chp z5Z2|F)+%mhvNTL|w|==Rtxdkpj0t3Tp`g=ls5E99tEY^*EcjO6)&;~afBlAwu->RQ zs|#nG+qjMjTSCYO3VQ0}Xi|#v?(Qc|UZi+|&&$?YS(UGrd8(DDoYHQtduhX4H3pti zl8@FCjmn{;@4OT5I|{1_nny|8{)3DRcslb&O^=MPb_t2(#aqi#4q~63apK=>4N99V z<*(dCq|@rRtTO0rMs#l7zy^fe%zvH3A=`Ue=Klt6tmdv#H2rD6FHr#Gu>BhxCFv8a53NO z$SC0I1UlrQF2J|q49cXUjhCBh<$6O@>^>{n6+I4~x~s-xkL6Q3DU9qdj>&g?fnKL} z$Z91TjV$SsCN2E3kUwr|zw)h1YkXiLXP)7`I8O$LbF)rd*H~sga8E>DrmXA4Ul(W` zTu%_|6MD5oL{G4ILTtuoNRdfYq3x%JZKJy9yE&W`uKcN&x(=APHP;q-*ZD`dRjA5E z$gRH-o2`DjxRt}#DDNCQt;W5SGc`68jG9V4YH*4u~e8F9tt*S&5mqdW2>Dr@`;E*YY!Lw zP69Azl4zm8*iRCRBofRxK+&Z@#8-j9NtXww0Z?Wb<7c(8U`GP}jT>;e9)hY$|nhwZHqw#g8L@9v0VVYSa!ev6S zg`?6uLe614|L`7Kobx7+yJhEeD({+e9@9hQ@{{hQ7or(dP$eUx1Jh!dliLq-CJM&2 zopSuS8)o1aIE3`0D4@1II0<&XuL^FiS1_A(`9cL*5Ib~gIvjYqfCcBLl1?1$wGf9t zs@nol>O+k!gaX(xI+_bTLI%ehfn;YI&*wy_w``X#a9kLb;hpy2;ke&TE$b`nVKbr zfi{<3_mMBTwzR}3Ggxidap7k`2;W?dz0@&mRGEZ=)JNi>QXcB`a-Kr*n|bdRd5O&}&$>Uve7kEVY0D2|?^ieM?;#V_RnFAFdbMwH$(37Q zOVY%hxeRIjbS3zwkTWWxZJIVJfRQ!_DCE6Goy=w@@M8Nna9Qq$aXhkRrtj4%*Lhef z0eW(mUk%ic87TXqVSwYi$x^5~G8cmKGIR1xV(?-~l!=mYNNV5cCZl=xUK6{Kz1S&< zLaFnZa3}UstB+^h8l~(zUAERBU2-FA%l+ZYw%}1)!f^6#+(Mq zkTb~;<3C?GsxR^c<>V;MjK$6{*Mmt{;P^I5s?5{vH!-(+Gc?xN5?$6eAJGrWR*Vrc zB~s}n+Ap0uX?`+gMfSYN2%AZDtq$IdcdAL;2m(IOV0RP>D0T=$EutsDyKven`sI)5 zu83Q)n-8h?dVouIQOpK6+84$u{^*|3`NVEj?&!35JJ|O_caDkA|EBc)j3a$t z8?X7S@25;$n!Ls7cLc(uQu_X~ez`MQ>T<7r8B+)AGTyc;C=p~wN7z$BTZ^cPi2_Mk zx|r|_)5fV)UM0jd!lz9|w3iV7D#jegB5%pPPzupP0e8h=(^H+hSehd!n1iWuy}?XQ zLNC;!%vzwx&>-4Y@{i6YM)Uqdgo}ESgmv@(mm?tn0DlD_XaVNd-v{uw`VablvH%3` z5Aptl@c$RU=8KaGi|*&zwFXI08DH?vI3K~h66`&G>z}aky=AtM#{i!_SffYEU?bL$ z*?sKS^@O|8(~Kypo8_BdDxosPo$-Ez`aFAEHeNe)WUBeXsh1s1?Di?hm6c#2dwb-k zBwwTwac{>PQhP~XJ2`~}VdKfA@Y=cNOS+H3mn&XlS(OJ4Kno&hF@KNg4uLd-?|MRs zOD!`odtI^PBFOQww&!Bk{BN$U-K_bHDOpXW#sB*B4^E2$h3YLD}eh zT7?j&`v8JuDWs_-+#xX3EG`U)x@*sq3521UHj7^eIC4M1mn5l9usp2Ki2!dpUOhmv zIQ-Pg2r?eB!*PM!+x3KaAUrwN#6(V|fVXVjcic5`5d81f2f@`^f zwt!&hIN>oqV6AKw?rfQ6NWJu`T^UOxu1rz$*|13CdTY*Ol7fJ4p*M-aHQFN@5p^;w ze2+=hLFt@4W&R>2pb9?gqzc|-O1=+l!8_?;N36Sci^e}EwzEy7auLt z3Mv8~$0^+AC4gvKnA8f_|#Cu%w9|@wK5|@_K&R)`Z3sB z&rZKGdjw|!AA+IKa2|)8moMHamd$dHb^M+`(ReT*qmE0Sq7EPiF34pR}Pcu3F=p1$hBFbahrPjsjhK4>QG4pgNNG& zgDFOUV7T)Q$Q~Uz1Aq?SUmX1`D9R|6lRLB4G`kqGaNi(+9z~0S#O{riQ71+ z^Cyl^e5^5Q+@?d+ReA+aTQ$q5w@mpf2>vzpR{#bi!j}Ji0Dr3hq5mgKK;Zuf^CyJ= z%Jv@`zwZDm`6i^6r)3ZrX6>5^Iny5!wZN_)ibI0jU_bk|P2B~=Stb|M>2pG=t3r8J z)Qt+ASe>uy?6iL9YaA+(P*ow26ek*ef0N2H^-Zq8RQUQTJguUNx9GrzB`sr_qz($B zPhFw=!<5LVQO*D%<%7whrki=SP64^_vPMP~&FBG4y#-#R2Yk|Gw~#a)DX6~u{3G-l zO)Dm!)25){fK7#&LC*wX!*~2VmKe!+Q@mc6!lEj*oCUB1j^Gp!J|S^Rp3;&&;AI<4 zv5K|ags8EL$W&DAOR{8NlA1BWbJ=O+{yXN@jD^BHGTtgv;Zv_=N_aiL9(rJEp_sMW z8g}QIT8J>gV#D5Xc>=^nO_)*gkglGZz<2CmdSVL_x)E(jO zk9fBbi?x38bMJeZKT6>jA9|&lp=F+DvF}6@=3e`NE{b|$y%!0?Bxt!TjOmEgXB0Vo z^F^pn;>Yh1Z_wCr+~v@X6bIi8ej48U=>exk-?eG=b|}~PqrsT~9T|roUq7PVWz=C> z84?w&s)*UrWwIJTRNRiLJ0K`>RS?)wh^z7DQf2C_ogr`}LERl_B72U-zU;Gy9pi48R0DXME0#WG?7PXUDC!wHch~c% zrQg_>(B9Dm+!Ii`WZ#Uvc{&a_D~}W8 zK*^TL`T8yIZ0sizKJ(UC&K3>N5->`nib11SBWl00xYfv}y&BW1ejjk~uez&xEpZB? zzL>6L3Hw3xFHND_qk(N-se|+2EO+1={=fF_JgTRC|Ks?#L4%@!ibP4}m_?CH)oC6K z8A7CakOs|$gd$^xB12LVQjw_)m4qfl#!luki_AG9zi+N{?p?oi@3}wc9Dn_ud#&$k zukUvcug_zz-fK^vywH%+qvuX z#GAU=S5+5Yy4ShTXv+Vk9{lkh;Hdc>yZrS6*wP0?_eXXDQTSHqek+9krtbaz)z<>p zXOMqoj{k<-pVOX6*;?kM8aejW+c8yg;jS-Q!?fq@p0B#)>cyI1g&BQS+J#&!EwvNL zkw0`eI%Uk;Md!*xS6eQ7-#RUFx9+Tr!FpGoN|k-Au$%Mbx#{hSF_NqI3tp>yI$yPKQVhzA3e?K zX6%~B&xY)@R9F}1ce}9jHNlQ2CS$A{?&l8LvDNiPw*NELFg?pTC9!Lr9G%}KzJ7FJ z&2Vq!(bX|B3m)8il^kGe^Z20XFQ@z??(dTJ-{@X&{?5Fq=PnFtQ&}_a)cIw4QtK-A z+_H^McUaJAt8co<()w3h)>$srP&*obe%r+!VOvZ~2Xxt#qUjU)(pmA2_~T*6UC)^u zxb;TSep1!o;@trc2A6)0nd0fQ)OCJ&)7F{>*@*6a3O{{ZvL#xoVS+_(og+T+>6*In z;r%rao_w9Wx6{NY6EY@7VI5wWABy_;t-_^t(!sjb#s?);is1ju^9a>e4AS8=tBM1T0E!_gZgHo7P@_!ErN{ z^)-KSdUV|Jb7cAS;FI;u9eoxAdP-d0th!RNbYsbt<;}v&GZnG=5(ajc&v*JLCRL+b z5v-MPSA4T2`o|(8(rxu5 zT{O#OUQH^S zHQg&jY1Cx5Ae(tUPGR-uZ?3XTOMQ_W+~mG+--DO=BP3+oN(v&vEPJbzzEvC+KEkH{ z`iINUR*8S^m{j=i%8Jcu)iquTovZhJELG3lkiR;u@=?I%8oA!n^y~M9`ntR?+LxBG z_@>LCoAaC26h7H6VLWA#oX5bm!|SSgn+i`W`fAulxN6n4Gnp@^ZL&RkMP0w#nXVVN z#Dxo9^^nr9Yfex|*md~huuLtxA@Wth8DCDQHLqL~nmF$4uRSIYC|NXA`dq<~^X7BM zkINl0&_~^@=T$|GtCD6fvf4cR{lET?M*e6Iu+t)~%YVH9w)6tg{gK^36u-6aw?gB5ViTbJZ+Wf&g>cN-uMFk1^b5j%hH-~R)d+OB*)el7( zr40w}-1Hb_F5`PIKz#0AXNwPgReC;NwyNfIabtbNxv(x%Hw7QF-5r-4@2!93>4cmM zH;-Br%cqVr?JDY{wB~YnP|sSWV&CXT zuMJd9ja0XV9DHdZWqJ1KhNKStf}gyNw$ew%Q6n1${(`)VxL-%gLV76_RQ0E zIazb=Oo4~iwz??2m^EsSk+(&bUrs)>e}2Mrlf#nbDbEKynd~5Tx6JQpE3Ivg@3T(Z z`A3>tJQzRaMbZ-AsR?sTvP(VA-Rc!3ymul?bpFJo&&Kvw0v}#`GF^}&_0wiu ze-jpXWZ<@#l+Wb}C)(QD$WD%t&r}?FY_(_3o9V*SNcVJNvxb>pN#27-S42&#Ny#zE?<5s3voT}wlluD=a&5FrGVD9;lW-}o#$m8JzH#p_VnleHy-bn zja|@B=2OA@o{cY7R(9T_VKZL+-CMK$Q_dxA|DtSPAD!>1Y^>00fRB&tozrW^hmE(K zxJpsacbU^?UFQwavvVCcjm$ju?paYJax>3l zm7h{>72Mbye^Xm3;OFsyozA{_-&=h`tYFjVKIvJl-fit-qO|1TPDQb&4)qG#Zx@{1 zwqQbM=f<_hS=Q3Kt2bT{pK7i?w5T*sR;I>h-!lEX57U3qKE6`3!t3*rdk@toZMKSU zT<%(_xHMI*vD2Y;7ma;4Tzb;iWMa_T&)HUyDxs@pjF}oYwZmB71c{oU+XjOx(oUS2 zD%#epHhENBf{tqNmhpp)`$%XOefBy1M}^?`-~OloT1&_b`RfI+r5}jykL(E2?ptGi zD}=wd?ca=iEr9m-r)kE8-1YSxo05Gd*ZaC|iIrum z%f(eiSL`@+s5nUXQ+Z?F7z6!zll6_{Iz36;(xitf00p`nCMIWah)pCHrfp-DVKRr!b zaS!=vOZAr|jeRVBSN!(;s{P3!cS}Cctu)HY9u$7uro`0A?pEl^#jbbXt_?}aD-JhX zo;7df4z1eCK?w^2EUd;&UcI5JdZyHs<`MlKP0T(rbc5ZUGdcE?rMsCVYc8MddFY6} z>#5JZr+3^i@b#fq_WRez+O*D3OL^e`aKi54k?)lni*g-3V@pK0Rx4~6a$~;y;NCaG zGGEJRJ#v>Gaaj7HUAb)Y>EtVu7Ml$BtJRr){#Bz^*5x03%hjSX|WxG$EI&70gok79s1+&vud&OtSujn$vx|^}V zo|56_o3?11TU5=mw}14udwN6cz*Miy?yHYRI_uUhPuy3&NT>GP+?ZfJt!R7yiN5ho zMgApQf=-&s_c0r|W8kQc=bZc_bazUsWz2XffB)CXS#$kO^FEqLsVnu`dbdPFeOUYC z#^)D0%@ZH{OF+z?wgWsauU;NC|AyV={tp+8u(SA3wa!-Ev)$oEgyc+<_@MT2bKgmke!6!_D6HRVMoHViWFZZg38v}CNOW#&TXCZ+nWc2DK* z1vjWQJ&zx|*rB8}&e8JJDUaUonpY@Je=Vvy{-oi_+I7zFJcis9*{LhFuy>yKi=#7l z`965?BC4+PfPX}in; zw8~}4M!BwmcW3)1haJmxk(j+yL8|%1^HvIr-;dm9nUZ|VuA9~Fvtl*&{^5fSYwPqy z_ZJoyE_WZke^9rmceXPI6nC8WA#A}&>E zyAP|3iwbg6jyZetX8+|5M=quww=xy97n!jrdP?@J!CLKKxTT$pch+b?y0_e#7MurbCl(}4LhZ+_NjPFOktVoQT67dZA&kHcB+Xz zv3F;&gvX{qb(t&M_Dp(JZj(MRGO|WdPGYC<0`C7|_@e^Y$FNJpUoL>cN9_v>;E(JH z(*9dBe=CH4)atJV@OqNKtakU>Sz8aM-Kv(T&;KOdvE1Spxx_}ZX^UgewwAN8eQGFxP8Vny=Bvk~G>TaD?bs(*`u<(R<2x0e zKdh~>>|i(KxlzvnB{M?whMZY%<@!$P&A_Dt-W%mK<-|yut_(j`U(x%++1`Gp@>+dD zvPP$li9h-1%CcjdiWlbkrgrap(j#Zr&An3bwSLcLd^))=!eBzj1&EKFGkgu=|zZDb-Fl);1*_iSN;1t7HFb(wR1KNgp)U ztv+7c{p#W(?VE1{pEx;RacVZtKh&J65^<%6(vnBPbxCEFdb`>hEbZ($$8L9m{ew@9 z2bPB)v8naAxN@!R$k>U^rmEtRRk_A`_OCiDoI1DqQ)xs}y2Y?9Tk0$377li~<2R_b z(si`O;F4YedjrNkvGeL*pI!TDvswq$6N}f3IIZmA+M`=^{+WvjCud7HCk~up9Tj&n z#wlRa{5wxgY%YjIrB6(j`I2!V+<33kqpD9XVuMy#KUU6^SRVi6`55V@!_ND&tz-Q9 zH`HAVJk_hSoAvyYC!)t*@0GafWUqdD$47kLHQH@Z>A0uz&LfpxI`@h$xGrXBEk49S zW?JH!$&zUqT_(34GHXIkVdc?tGdIks`JlE_^_|&q*Hc{z%?;z?YR284bg-x1=is@M z?e||(nWVa1uj5XMUc>i39^|4m&j09HEwxKEuVsrxjARU!Y+T_IWH(FQ?pfKuUWZci z`uIC_&3*DQZ1Og0A2dI~rE#f9xaGwrtRPBp_1ms4mvC6UJi)7*Q5Iumj}nL z>6zJi8C)nB96JF#i+G(}wg3L}^t#AL_92OS2rsW+V|9$xP1+b0q7EFO)bF{bc zr6O$xi%9KLZJni>^<2;@I7rYpAjqq)OQ4IJw?|(Ww-BE&55HM`jlYiAjkWU#4tDYK zFb;M12^keIZnnSMw+JGl0x>H=YnPTEOg#caysZT4el8I{exZJ&Lj7Dlf<}9e2?}rz zb@K=c9v?71$i>aqRv_l*5*Q-rW-DA{zkjT;-*0Q|7aHO-v!AW~gr8f242=jB2)=d& zHWDW6`tK69OyBpn>6?tUb&TEGASNmhu@cA%b9`OOIG;Hl5kUf#87^Tieg84NkH4>n zyQNF8x1CF%UeTzj@-vH?gfmSM$hf$<3EMh4$lW7IIFG_V=lM2uE6JLZr<9$m!~~+| zg4QzwT-?ot*#m;C1kxeiK_0>00W;kr0!JEs_304a5?(|2X&)l!|9w4e40VM~*7}c4 zHt`8D4hnLaZQ|qqbydQbu80a+2{(nOPf&1(mA|`3gtegUOqZ7NudNZz(DFrqznA&% zQ&)*v+$3ji1hf@_TLVu{^GHN;Onu&*Bc;&r>htcKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY** z5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0 z009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY** z5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0 z009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY** z5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0 z009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY** z5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0 z009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY** z5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0 z009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY** z5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0 z009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY** z5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0 z009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY** z5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0 z009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL zKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~ z0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY** z5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0 z009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009IL rKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#~Ee}2s)PVtigu;PR@D`7dA-<^(yG1!F0j>o zDXl6j;1XN!NNH7L0T!j{uozzFHlRAQR zQg5$L>eAIo{kA$g3FRM~I$5>ZZ|9#+P(c5Lc|mXs126ysFaQHE00S@p126ysFfbnm z?0}ia{|i6#X)Vt-g?hHB(?W`;ObaQVDlMdVinNg8snJ4;r$h@Wo(e6bcnY+T;;GNG zth^vQlAR_kJCa$aLB^Q%8DxxEmqEsu^%!J~S%*Q!nDrNAj9GU<#@O{1^t;91UfsjK iGtw#_dYk#;9xlRJl}H$X0T_S*7=Qs7fB_iL27UpH&(z-l literal 0 HcmV?d00001 diff --git a/activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MessageStoreTest/version5/db.redo b/activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MessageStoreTest/version5/db.redo new file mode 100644 index 0000000000000000000000000000000000000000..eaa3d6f245f18c542913a041200609e173ed6b6a GIT binary patch literal 32824 zcmeI4$!-%t5QfL`0)!b%AaW~<$QJrK%KUcAzo+MOWt47Lb(U$n7mh#ntKUQ_+QR_?5P$##AOHafKmY;| zfB*y_0D=FGz?NN+E(FNqr}jMcSHh}*{C@tf>jUy=zL1{!pYAMcmMa-a}^00bZa z0SG_<0uX=z1Rwwb2y7aGtu(>E6GvA6s@gm=&hYJjXzA6bv;Wwgon0QdVEq?OFPWy* zhRgQ+is@C;Yo-b_LgvTJ7Tl-npXAY5-TWA$- zx5kzQSO36!?hswMh1a7)uRFw7ZUOfA$mSz#v00bZa0SG_<0&5^pO*8v6L%(l> z;9t}=d>Wh*0uX=z1Rwwb2tWV=5P$##AaK3}YODjCFJRss0uX=z1Rwwb2tWV=5P$## zAh1CM>S-Oo>H;cb9bBjjh$-}{k`AvbsqLzgrmiX}=chx#(Xl3^ufMIcqOmAfF^aE3J0AtfboMvQlcN%Sx!7E-RgO zx~yc{>9SI3r^`yDoh~blUS($n-ODItYKmY;|fB*y_009Unf!{*=iiH3G literal 0 HcmV?d00001 diff --git a/activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MultiKahaMessageStoreTest/version5/queue#3a#2f#2fTest/db-1.log b/activemq-unit-tests/src/test/resources/org/apache/activemq/store/kahadb/MultiKahaMessageStoreTest/version5/queue#3a#2f#2fTest/db-1.log new file mode 100644 index 0000000000000000000000000000000000000000..60d259be66c7acda802ffe84d274bf94bf1ef7a1 GIT binary patch literal 524288 zcmcHAb95!!-Y@((9kXMrW1Ah@wr$(Copfv`9ox2T+fHup{p@|teeOATzwa~7-78~_ znrl|Ax$67J{EVv9`~d&}U;!!1i7E)<@$o9~ivR!sQ2u%YG%|hp{H+ZLKnzJJAYFr01-sN(B1)qUQ~cx-^9+;p4!IH&d5SnkJ`doU)O?)fr*ixiHeSqftiktj*f+% znUR5tmYwd`A1y%=#IJ$kP=6a0007Vc5a9Qrel@KC-2nk8|0g5<*BAhR|Ht-^kAJ!O z^&Q}&Gu zdmpxl{t0roqr1oB*$JtwOfW*xqWFGYa5&U(^~T$fKtpBQP0brLS{$BKW2>-i z@?q>uLdj8?wIj;6o%L?XN9;g`2U?8bQZ_S7$Z0i`P%2ox3d&KQh^ELwgVvn`U$E)2 zJmcE|5v)(5#LI&FRw0RBa5hgEb!JC+X8TDKDTYKr%VarrXt(j&#|=XJog<2KY|FYl zaX0*kD(?~KqQc4`KelK(M>rM@@47vy1Yjq5yIf9C3MX=5OgdMT^mx@8PJcx5!s7tf zI#3TC3|)Bs;2f{F`_4m_(*>pW;z5iGJf71X z)Uo#se7C(Yg+ao7`B12ie-LD-dvY-4P-biotX-`h*VBE!75&_AwGgs2Rab9w>dF$h zHThAECDkNZ)ByM0&z?35-qO0se~QUC*@dp@#L8CQP*}Rjg)~$xQ>ii-$gbUwVwbfB z`;>7;d1sa+IkFntf9717gjWnqJA$(MveODKek_{rR$Tx{a2^A~i%ml&YB7xUC+?!oSOiIVntL1d4Xz-`` z1wtJt1I(9=%-)*u%CQxk^6g3_cY>r|JV$1C)1!Nr^X59o0a=Ul0RD0VdpN-aCIs`( z5EU)*xtaNFtRR=Q#-lt1AFAv7(<_*-TO$%E(2LmKwV@+~V0~%|Th-R#F?zNYxFyI6 z*TMik_TxxY%CRNEdh#p!{5OJbtEdazVV!y$XDC@s2+jKOpJU6ayET5~stb)yz$_!X zh;iIeJjd$-3&GzC1<2UJ8a5R=O*pj%l2}3bjPbNlS<)C6D{{S3^5l!n1njoKFM-k| zd?aXTatRpg@@)xA%qA#3_TD@SD-sF8CpWRnqmvRTKV#gC>-gR%5wqbW9c@7!iZ}GH zT+5Xk$!hM_H8zku4hB#9hzG|5`&6Hx?GI6)c7sqw^IH9(?lYBkll)afkVHk(6p!>u z+@5qKW;o|^g@7otaLG9b%*E4MuMi~A^t~Fp8Z^~u2oP}sv@%{3`0B&OPYI>3+?ED* z6rIWaz8G+)`RyIKq?`?IlK!Oxf9@m&p#81+Z~417g%tXa`J?|0z`v0{pg)rQ$>E>D z{@wO>{&aH$fP)-kt2lt5$VNxUWTsCzPZ9dvk=wLQOFc!xV42ky7`PGG^!U789@vw* zi(~^Sk1g?+4_(u#C3sq17QJe5UsIn}ciH#DiX-T6x=4>_23p3FW^rr4kWLjT{k)8{ zLCj+g=v2+@H5i|seGh-Lx601pBRkaE>G#*^K__X zbxzN1sSbHBqZ<&+!9+_ZC~rO`_{v|A0=nL@wjZvakT>xV}r z%xKODjtC4Qb?=&mTB0>E3Hx8Z`ne?!m@>(g85m0iA{Pa@h=qV4qCT7XfzoiiT)*nE z_R6FWd(VMhIMCUPev8exOB860tBgzGaE|}<$*g1wb5LLWNT%P&nYY_v{ciM?l?+-5 zk)`)EcuzKG!f}H?>Pn+|zPiV!ggYKoaoa^EgCrM7v0Wyo9NP+q^Slxn=}umuCTY{~ zjVd(!9^xoNlIMPbH1QJpT>#=4&0|E*@oIXVwWLSsoXg~ROtiIwX;E5?Tb>aSqRzSA zIXWS;)?S*1bp+_HD-vMt&D+_gVU(vS=DDp5vj;>2a{kFkJZ4=49*vhFfjMW0y~%ow z!B}-i3$4l+0#oA^)+rnL!D?WnQSo&-4lFSMWJ#9TS{%>3ZhzxIN|TIG0ttm4-vQEH zoL~XO>uo(^59W5GT7y=@GqOh0n-jncJK^|4Fps>SS`R75j|-x3fRQq$_mL%sl@kdK z*z(I!GzQZdB)JAlo4fpjws)zV}*lD!e91hb$5c$ApjpO1$g?O^bFwx5j$l{T_| z^Un%7*#%V+0JXZ=i-rJrt0M=g*@0hBgQjI*EY;~g)SQ?jWxiO^ovN(Ctl3Y(1q@gE zg&+JN*Vb(yZt1f${NPXmw)p*G@ZTBm-~0ZGzZxCVng5tShF|>A|Bd_s|B>fU2LF4! z|NiNB{!U_ABgTx$o+s_G%50`v(wNjb9M~-?YRkdXSqVI6Wfyx7u_KEj9Adsbe_=!> z(kMHp6Cc#;6d~DrmO~--&;>ZIyzalRr7uJ_fwEHDV}U7( z+6HNQ*%1dU#XM4Kxh>g)6snyN7rq7N_7LbKqkP3prhCW^Pmcs4TV= zoW;iErnTAsk^SZL%D#{XRFR#&E%7wMT!TOkpzvrcvG!edw+v#!##|GC%HEX8dYRkA z%olz;Bla4gBdh6gSukD2#C!9FaN8D1_vmhBg!IJ*C*q<*<$1K)RR_Zbm+KxeR5~6l zP5n}Qt+CL@&KN3rA(KdQq-$lx6_Pr;<%GogXpvVU&*(X!6MzVl>@+Eo&qhkj<2?4Y zl4i{1HC+`DrawJOVU8B-25G^85xRRH$|yaQg`DE6&FvejjcmMU&0#N@)N$4laRK*) zFkE9=<;;~8-L?*u@FVE-qr?e!f$Tk}oVv#Qsj-;>x1C?$He;uK*gN>xxPh`^zIH+n z=f?ym1=euza#;Iz0kPgu6tPYr*2SWCYGKN-CB6rR+>wLL4*6kyYftO}nrFm;#`_o2 zv|dqV*4%l>pB}m$^HuN;Li-3|Yl}I-rdv>72JEn^Uj3iUxB?aTV_9I0FKB->cE>MC zDIY>kqz7*{ch{bR@W3d^Xgc3Xej3a$mR730`;ZwxdyaInRF4*qzSZ{!IpQxDruAMQ z^&UzuaNB3CAKSeniYgT`a4+0l=gfr`5;y@x`-y?Xv)LvSu7I2K>r&3!?;G$HY&7Ta z)vipLo@_NzyGqlnY!tL50U2ps>9!Ju@T}$<8*$yYpW~1mN56Vfq1^KNHp|M52lsj2 z@AzYym1ETmT!a85tm6}^wIt@6HPe8UuPTxGSwnZj8<{b6~w3*BEYl`rFZ#DXa1EbBum=_KB#r?+Azq@zvwJ!*2YFHGgB@liVnq z9IsKeZemP%$>7QhoiT#dUg)yEa3zK;?pMg43QEhDyJRGrljXTdq3lH+-as{QN+$M& z>&ZRXY_v%2i&zmRlgX>xLGt~ayv2P`ld+Mn?M`Ob`;3#lun@r%3IYx-vSu_hojF+xeA>m)5Oo_K?Xc3y(+io@DUI15JQ$ZA3umly{mkCOHw`#+J(CRdCr9 zOG|I_5F8uU>;}+OGiIh6@v$Q~8L$V>Fng6{oXfTDV_j<&6)6FsB1N4+ZW8mUeXGHP z0n%38Rccma+@liv`7)Ucrez#=$LIUOeoK{n&7kt4YUPHJZj;fyozA|RTfLk6>6p9ap#aApY1Ut-@%l^GP8w%|3IZf#vP zk_cvF-VNFOg$S+GdXD#S#Sc(}y)G1bNFZHiihA%xraJfQ-MaedMeezCn~n08H{q~S zxUx6ASYkxx!ex|!?41sVARm=A8{>kYb2QfK0)SfE0v_7h)-Ov;@h+o96!`u0gyB&^ zCE?+i5~A_Ait4c6)AT@cvVjF*IQkFLf|es72(QK))^nnRhdr&hSy^*+r84Mlqj$W~ zWErW*L=(DS#bc>+fwuk%f`9D$EB=HLQD6UK{+NF8$M`q$2l_{pKNCXFe$?@AlX1dSB9)vI!PUJB984#yko2VZN?6ePaCHrYQIR?keqRBf_2838ED z1fp&preKh5<)liBP`4F*C{CeDwb_%BXJlm_7Fd!yD$U`HX!c0B+V{}lg0c%zWv7#O z>R*Au`_zAv^&rhvp%1j>G99XR^~}$#sQxL4ri5Q?sW{SpEUwaL9|pY@LxM6;{V7Ac z0(C2%!=Gnhlc)sa5*;p=Sv?Q>vj}3fshL#_y)Ny?Li_WmuDY|sUT2aTVe>TrT$M$R z84UJjLsk0KgE%7h(qIzm@n-C*bj4^=k?>~&!w1cEpSdfve0JTn2~M9VZORM;WkWXt zTi~Y%bQebQE$yHKQ&{=}wwjiZ9G8$qt_kfP;x{1T>}BvzS?De=2uBS8O{4l;QI8Xg*KugjZM)&D+Sj`On>(m(_=?R^#LA+&L zkN2Ub2%xpGEd4e3OGIfisNncEl#m$F)SET8`CH6Hx};bER1L2T`;}8w4L~^7Q`c-z zY+*?YhL*I1WUvw|0(>}>Ug)+46OpRbKn$H>j)=mUGv$HxphE}RP<5)Hk3=9KjX4ZZ z@~pgK{n0c)2_pIlhnT7}<*#IrB_10~c&%*{h;}($wEh0))w?`ur8PVqzg6*(a|9!m4 zZRKvOQAIpaBoa6|M=A0^L(LG|C*N0Lm7HE|Qg%}U`qkP{r7eOyzETT&67krs+p91g zMkV;GWAYK1xv2Hc*b1Mom$h4Iqs;o%1h4R!%#i~6t=+L~gZ}Y~&R^9DzdAGmX-Ytl zK?ijWjy0R;<(*th*aSzbjqPn@NbUE0vO8B1kq-hJ41pay-!FEDD{>AY)$SXuf0+eV zIYG91T zaO`U7PChr^;!3lfSQ;~k%X5!D=W0Z^2_8Lu5Q#aefw%V@!7zeKr)9tAzj+@8>JOyw za4o<=>0+Hf_bcD(@d6LOa;=4=g_h&ZSs*_m8*(Y(HNUkqkH%y5yT%myW>XlddDOWl zd1Hxb{q2}pM$NtGD9!!RWn;@L{C4RWX$7)VXJ}LY$x7nknV$=r9tNV`7shIDZTjnv zLoS7J%VSZ}lhdI*@Jm&@iT2z9pKE53PZ&iMVbTH7N7V1>GW`ug2!=B2g+V_#zpsC| z>i`0dwFS030~|{X|C{;$uU&t|AN(n}?tjc5^Dq9G{zm@5{>bzvgZ~=wpFO|x$EhaN zKACg6JQdSX`YI@%Vus|h^IjThWfHJt2Dtb7a6_z9<27X_J){WP%Sb%dEe}7}WIK?n%Cx_Qbq#^0xWg4x)aDm< zl&{eJ({V&gM3CdQoBq=Rs0|qnfz5)xluV7faN*Sn^N>2wh-Y>z=osy~h2tsw+cx?` zl3{O2-1<{O)!6M+aX}`etRs8gk|7PwVBv0M?ZVG_;S#Ex*>wV^V5W|Nw+dY>>|S0_ z0&HW70FD5`@}J0Yh_u$oBOkMDGy*+xJWmUEwWfifVBz3Xm7qzQd_7)UhsVpiJ|osI z?nt!^eY!Sl+rHqFBW-&Pi}_F%jT;G9QSl7z-0+=!Y=ckd%{avQ=<2fe&Z4unt7u;> zd7}ptRYbG7_7O6B&gD&cnmINBP&b??j}Tf+VOmG8iFR=NCc*fZM;cQ2T84eNsq(ek zpW7!Nnec4*J3-0B>R9WdGDQbi9M$bHX5ClCbR?VcRtTKf%>emNLZ+kN4F~9*yzUmV zrhoC2ELX(}#eBmbSg!MGn zOWc!6S8xp#*b-%(VQx#DbUT$D!PX)> z$WJEg3~Q~`Gx_>|PW!)T{S|*0YAAvKF@G$-_+$PX`2+tW+@B2oMb_V*76Ts*7SuFe z;=5I#T-bm3LHgi_lMa$Y4y}F)YbN^5flD_iuBp>b&%%L*p%O7Pf1YFBG`kQH2UuTd! zMyq9b0*zv)3>xV&MH0D82E4j9ufMZCJ~hhEIJ(`_%pF0KqUjb3E{febtXb+RLfYO zo_bm%ms7J$6yY4qEQj-H!nboK`9FR@O+;eJ;phhr=A2eXYJ?27XtpK1#J=dG1mb~} zkcb%}PutDW247Gn)g_=hpiJ6~c56Gk$u8&YRVwJ0Hz0-G_b{f2|oQ175EB^E;}UhVLVEZI(SJzOzz*Ci;w7OYYf64H?~wB5XEH4X}26QTKeZ zck5zqanatvJTAs|y`=_%>#`KWPcYdX50i)A__n!o_#G17UMX}?SezJQXbUTmj8%JW zbr;wzsmi04jOcv1!gL#J$DzSDq#BZPuX~$k%tD$XKquNs z`_OlFFC|)4Ky%}!h=`+m#W66qhN?o+aTS4(pFJdAMD7wF?}??bu`^?=yDo}d@s_Si zvYvy}Kt{h4`tnmL&jHWYFJed-DmuH=-q1hj2SQLlsBO^$;#5RvW^Q_?_R9Atq;Z?* zk_c5yo>*lx^1p)MAN&4_KUi24?EjcQ)?fUw{Ehs5`6J4o4E`h8ziIr=AG4GGfFlUi zN*f(2uS;%@8>bhSVMcPjlah|)yZgPv$EiC9S4d}GUti~xbTS}f!K;2zPGSfcezg88 zZR_?opg|-PoZ9IrLrFYafrz~?wKNEW`mRg4iG-x=5kH0`g*x{hg?(<4t-{PDY&O8} zkm2o`6feL@Oq6=~ALS2ZFs*KG?PT({)$Z^DM?hKA99tZR+A2zCN^dnGWH}La{o-Jb zWf#9P-XJx@P`i%@#ukb!s&-Q#XK6PP%a8JgVba3fk(snE$Y=*>oolicn>g$X$VoK_g zp}+GEJFD(KUx`{-=-ar;&QH@4i>U~6{z=Ph=1h;-VJ``p{v$P+M&AE*VH3^P?x-ZB zH$Y~Oy6hEuY}H7f!?lS`)P}R|78Usj(Y;6JFz9)$secaNh5w*Sgz3^-jO-&T@tZ#9s-2iBi z{pIQ*TzASxQRB8ncky*9$u7a`#ALtLnJ;)?ux1%@em-RvXSeefU1t;|vJ(Ak_oiDT zG~0qkbYLPL;SZYryBQG65XoYqpLO<<9rbOXy5r3bF2&OLui5!7m5@tM0D4Qo_Kl;$>YR2m@X*%Hs<>*= z;;0H+HSS_=LeiIv5r;S92eYF~3E`;G2Ij^TIf+Q?7=xH@SMVQ@`X=n2`H|UEo=L9E zRg|Mnurc-n<1UtGV&&{pzU)t+K|-)DZL;{j6(YYNznc;%GR;N)3A7)1kVFUo_U%C;h_F zJO*gdZN(Zy(zVJ zY5yK8DYp5l_vK02Kp(Y;0c9pMq(6RSz`=fwfFkQVSh&6_S@20iT435O7JgCY54jaq zlPYP#a98Q-e>4C8wd=3=>r?!S@gMWY_KQE(zmdN`pYZ(2;J-%vXV35a`NM^(w^oQV z36UlOj!E|FT2Vba}Y#8L?@EQ>J2N-hyY`Yw@eS*k+kz@^RUn^Q!b(mM568!>` zQPjM%d<29{_H!Y^SaC@bBH%Oty>E?hL4g~D zx*p`3BeW6NEWuI*cffxZHJt#2je?a>qhEP&j>?$vLj!^x z91yv?u2aQgB@kTalk6KZ0F%QHF=pKd>9=shIjG0{nPeqJpW&0_zowix$Oru0NIIBw z#|dCM_SLKU<_fsDv47Rkm7Prg)us?L0HFQ7{*D>?=9qmxTa@|{;kg17!b{IdPAdb6 zHL*B_9(2eQFLnKmZ#!z$kE4{x;j^t;;O7`9&I!nb1rop;4A6jMs$SM_D6T!RQO4Jt7yhc44~ElG@e2D(n0rxwhr- zdj0$ltDNf``u67fhB$BsB5*)c-J3)pp-2oQICBxUg1PE`Ez9D=Q5R*-9*s_WJQHn& zipGt|e#+JY5ykWTw^KGsRGi~-*u9MVsRZ}H9R+0@7}$9b^XPjxu4m>`*>a~vL;U)6 zWNEs=vJ$oMn9@@>PIhn2?l@7Qa(_f@iO^ZRDiX$}=LGl=$48#*b;l^jfpY<`NBVJV z$x)ZFIs?liMt&L?d@e}kD-GE*q3{yil>%!cJm@kafBCIq75*d^$B+bCA4Oi?qA-t8 zBVu&kO@^8ST$b#&BfKFBEmh^Z5weHp3Cc=V=E6&7SedH6^!p#jg7==5352wY-4(|h zO%PCy9g-Pp)Rr+ajBpA9xayv7@S3D--is&xTT{#V-Zxd#63$ar_)jFfKGK)nw!C5u_ z-~;EID)LD(_fXaeF*i(BG-&*)P9Oxg%n#?8YX8({X-sdTeE0@qu0OM$SZ|aZ4GjpA zsJO(BtSnlX%{E3*WlJhlR1gGsnTI+1IX~rrngd~y%7J#y8j9Qs1R7FCl&ncI`$XVv zNLy|Dbz7B5liscWZo9KqzsSszL}2L=9*Iur(mCS^QZmq-um3w^?7R)fy8N@{u3lh# zSSrw!B2B_BZ+m`A51x z9sG;1|8@7vl=YR}jfEB5qrOS3L0%Gbpn=~_2WTubBAVMJtBM}neB&;7OnaJ)N=@_R z2#M*rtO=u?#~7MaHd-BOynd?!id`z6u!a8H7U9|cmfpo!l)>1<%T3khT1x4l0dYlw zS7Si)iu^v;fZJa0s;X^^r7~+9nhC-M&#g_^281!R6aCi({>VJUTN$oP%Ir#$p?uOO zh2${B+7ccU5U~(w1KnmpLJ-N^&6f+$Fcx-d&fFKEfG5n1bW)y* z2gy|pj{9HZBZrEk5zd{)q(5CXC{ljh3WekEm^d!r-2x zJxH2ywP&{W^hrevq3{D2TIO#`x*?Zy_ z4aL+>Mk0GL>fGb=Yb+D82rQ97SiPK^TyrfSj6>Ky{313^bXA3dMIm{R=nC!{{H*qu zP4v39WN}AYd9szJIV<7s%Bo@YW0l#|qZ`VF{l_eKK0-Nfx{q3`!t-$xRH7Q?NpP-A z@FAEdxr5U)I&sJl+3M?%8M5PoQizp^<4k@Vdo{-p<+>PZ%UyIVNdt$ds~EnT<(=7a zOWm^kp4XECcA!{`;%+X2vF;+RV3jnQ$!73amLGht2muKpA$5-?BeYE!7{%%;9FhUfWV7g&L7DGxVEl41FZta+w|E~d3K#ACtz%IpsU_k>NkEN@UVXAT3ya~i3$&Hg zZ;I5bmHh(}a&@D!9aq;h_+{Hgth8K4?elJ>ES*&8ChMOcA-N=XjJFGk6{f4FEvc#J zVoW)fSjD}LI zbOwyARPXHgIgs3!;yeXXY8>8H8_RiqeSTw+?;>dJ6ycVm@p4Ma_SDUK?gb?Nai)XW zwg#(G7%!rXKCrZ-Ofe_-EMQPqmuQuFk`t_P7L-Jt<=huE^wDuDql*8{r4U_Deu;tv zjs0OSDYq2|WIra+{f>*n6FT(p_#tRmJl`~utipqq#e2|g%> zUTxKE5^a-O5Xe8oj@6n3n8#szKLN3SK_ZA_{YBgi+}sS9YBqZ{ObV<$UY$8`v2}9^F3%AxL29)RQ^X|ip*GbkvBY!NY8nyFNcA+B^qBfdG&bXk!DdAF~O z8n%qXAn0qu9quElyab!Uob>Hgz&x5_2B*=g#>waeliqLLPd?Oh=|T-I^th$F3#%FG z@lu_8^xIM4k$To-R=EqD=ip`h;BDKEN9G{JVl zk^DgNd^Tqa9;`xMS8waFDBz?HO^M^RY?kjCmCPipw+LkJ4N=4e?SQq0xWUxg=e?74 zH^mV#*y8)*vvx;6vkMXsy9wWCZrmGA>{X;me4fMhPYkclB#HFg@>9}$%Bj3*7CwrL z!bD)wm{2NhAFtIBh+Bup;P|?2iCgE==|;!*ll!Rn)5sfau3^ioQ+G*j{qi64H8f@^ zq!)DtsVD*#XG#up+4>yt8lExrK8xv*N(wHAuJA0iJ;>V@+E1kwB__aJ+>E6S_G|}X zYr`t6It;nUM4^r|(z?=Taw1+}LCE2xZs!pULBE_JbyX5qixj=-l5ga@)gBt0(to_j zk6x<)qFawpsbh_+M-?9DFq!Z-kZqvT5mGLEP%YCvk4xs?9Aqm zjGUHa8hAV81*P(ct@mXtqpk!2if z3`49Rr!oLIJ2L5Fo|Atnvq(yO!b&WjhPHDbD4O-4@Re3@A1t?35-!0h=&3F}+-P!2 zO7}#-q$wS&P1Tr2tp=?z#uNpE#f{p;7tN^p`OT#Fq1HVm9u|#+0}n{KCj! z5x(;{H<6O!UB4djECX-Kig5<&Zpht8RMVRVI;+XQ!no3GTrgX4xG!VS3`R_4m~+j0 z4d)QC%noH=%lca&E|o)N!Im29pf#ITS#Hqhw`29qcjwUyjUL-5qC6w)Rx(8i2-rv> zNRKCBPj)!>$jpU|*?c^#0GDNO>oO}mu%%VHd5s&&Ia=y}Gyng!>)!*2t@YcH@?Hrz z|2craSAP7Rr616Lr1}%We+~K1p5Fn~JVj}UqP85cz>me=V=5`djzfXp9bthKA$*1P z4fl-=(5LOZf-8L{XTSa7W2Owyf856fsq&N%)Yh?D?d;Ereskgf3>veA456ChFXy|p;6=PJcLH^Kpb*t0q zIL3O;GnHpc_6dOXeeqKGK%Xn?6OVws~&;QqNNF_5T9@;9^XZjx+ z`S!-~ih|<@RSf$kxHS5`b{aw+MG)zC7*?KP9w0h0wiaTrv-@bs2*@+OEK=}-!}#Vk zFK;ZpdBpKFuh-&79RhLJpwF~alcgq}?O3T(56g6iEgve#CPwTPV7WcMPOOb+p;!&K zT-VBC1HmkMR2nj} zxc)W(*ms0LbK*dE1?kKMA~^PuCkv6cK+Nvi!&fBj2(OX=fPL-PQ`OK%!`AnBSeug{ z*xxDRkHYUE2lJL|7oe~v5EUE1{ao_ko>XmrZ0Zy{b3K?ErLDl36a!jzkJD}5kZn_? zGIX72EDVc-K=OQX>H29|zzfr1z)!Z9BP@UxqBp0h+#g-R{borNc|y#4uoZ(2JSdZI zlV!6g4`Do=sTcU!5KHNQ^0UNtCuawRkDLDrx+;$-an+n$bgiD1tCLpVOXy0p0>xy1 zC)SN~!=N1W*-ovWrNyb5kGZ*&SlKD1271_WJtA>gfjlQjUi6>S{x4ep1>k>rmG@hH z2i@OU-~sbTgg>$Si$wo*n|tSV`L2u?%YnC4%HzANP=$i*SC>xt<7tsEm{wb*!?Xg_ zrM4xFcr$$DV|+^FVGRIZuTWvF9!hSfm8$Eet*B)u@RW%_9Aeex`RIWL>$!2pI?0+a1m_9*i*!_a>t5>Uizrs}7J*b>!h3 zwoApee}Znu_1tC-5_F!CalS!P_nf|Po!6*-dRhveJ;q@ugqWDWNGTE2^#bJ%Llp}- zz_ffIKtOeUgr|?ahM~nhJlYET)eNYExh{kRFncX5uoPUO&O8M`d7xHho^?X>PV`Ci za+G(=d5Rp>osb8avX;jF)YUS4@v`8>Bb3`JNW}od!pwD@6>%dc-Go3FCUNCc!qx}3 z>peC2&-Y$ZMz3A>jgDo{q8f=D&VhOXd zDVTV9U%kF+S*rb8HctI)AcOMs^lY0QX?zw@fXF-Ag;oNxu3)AI zaGcP%NkbVmJ-K*{-`tdaStn4-FxPsp9M9fBP-aZ>1+%|T?}IVPKm*%dh1_-vsGguH zH;(yCp1`6};4rQUdUVE6T^n$bA?AIxX#Wh$ZI&wYF2}FG)~S7XQ)B1IouhYjSs@wc z-zDv_<1qC6v4F#H*FTASMuRPG5Ucp>ceqz$^r($GFH9QPhK>d;!V`@_gdS%R=Q(-@n+W`prxj{{kdWKcDJ==NxhN!8V z@1S>R3b;a#}` zGh(=M&Ce5885zl9l1s7^pC*TEV9al3vh|=w>>=fwM6@*hdaIAJ{sibfrz?{Qi22F` zW|5V{d)FAyGdSGr>BbMn=&5w5BM`d@`YonXj^C$YvA5YIOC)|b8H6aEcgLXqgf>L` zNKSx{+wBBB>}gQ{ZAuSu$*^ZvE^aHn0C)liF?ct)~)&# zmSndd1~cs;`gd(Op^SWTQtB!TYgn<+SZPrl(D;T0#22zVEaJN9J8;p4Ntofdf z>!iyYW4DcKP8jMq!PU-F(U_OnzhvsP5rywRKbk{}woyPoAtkMMmpl$wN*dZ44F~>> zlHd0@H0Xq+bIvg^)|54FHbfwe_}Tt7=;Gi;Vk{Cu!NM{muht4u1b}2j#mdMv_!%r2 zd#FkT*!jMGGGC?)Y?Qvg~|hRE`ypzZ_N)e^0)?mko{dp~H(4cvq_cE6^f@38SG z9(+cz{D7lLCL0Dn&pe%qvJD8ySO6C4-cko*kL&Z453jWhToHg22_p0BUll$8=#9kT z-1uZ3oKQ_!T~Fgo1eU)X9N}OK#MX)AY7KjxN(h%*as(w_?c(BIvDd+4L@7GEI zfb$I7SZ%;zP7@`KE}+81M5MKggojyTBx#~e!>%OQgv65PvF;q;Wt*qenf_YFC9C1> zP#80+yk%0!sOJs2#?a<1Z~QKr^%X^|gM#GDQG$8FQ?9B(ImY+E*e}*w^db#m)G|OX z`*^3`lUzURKnfoL753WpEqjIeLV;OFURB=Tw&yRAl?sLw`jL6xEe(-@4Z@X!uH^6K z_4BArf{ZjO*IE-nVtQ=n;|anVem+y<2?HFc#BpN|1osIQ2)u#Ax|qd!WaH8^nPx>< zNqPfjQ!#2}H|YaYmwF$kyAX7mIZO#PQ9;{U?So?# z@%=j()K-G~jvdTOG-aJ+03xL5Z9y4-H=14Ih;%$2!LRiYp#>BG=|8hnz;?iPxtzcS zqlC3>nhxj6YrWyI9DNN+%Kr+2f9(5zb0zm%tp?rSS*!u~#|i(5-9OI%-!y)|T}3Ta zA))BPfEApOy5xUEc2dt-6xO_4mQ^KZ(P$#m5TMZf;gPl36+y0^ZS9=b5|Tc3$6!Mu zvHJ2r&z?dQLoZpWa;XLMvAY8&*)(XI6Z!S9z-~*9$`GB7-I0Zl+;^4Uu1c?&BF@nw zSz;kYyt%&@dpnyYz<`#s9HINobTu(Jm4Iba!RL~5IQCgUE?GJqwA{sLH46YltV#Kl z9v^33X4bpG5qLN85WXE_*ULGC@+DwkZXAkty*KJ?GT-ZniQh(ru6uurtx-GF{jj9m zM6J$@B^iBuR`LQ5@I7x8WJ$qZNgnR`UPd6q%3YTsqMSpBmiv@~s1Y$bvq6G|E~9^@O-{X`@QDzInw{>?R6r(BJASL2QKM7kO`r4&&Do!4nqTO$mp z|NB(Ek}S!KV%8JjJ2*gfx$h-~1;CIku?iPsUvorx=BgdYg z7&B{eY1cX8@g*?2jj~f@A#`5*$nKj#)|)UKC{QWz`*U@Og?v>Gv!)3E5Dm}!voC?0 z{&H#spH)Q;(9F_#<0K;=5{Hwx$2+pO^q^+dC(dJ%xFB<|xl5r*$$$%f%NIQQjM-?X ze&%?{L(1GIBg#WOLvkqsTfEj45xBGI@@ED1Z)Fi3#%`J{Me_tfN-t*TOIpqNd--t_ z8v{$dveaul{9?Vfi6Rg@xc&M|e(-OIWt@sQ!>FCV{?o5A>3gS9A_hT#*hQG+=Ur~m zov?Cbu8kK=lJ~VWRt5ss(0z&ja5CV>;z*`10LD@4@g^h?TBY$jo0Ps>Km>^>sP9?- zF}gnj1Xo&|GAD-A%-{3uMezistC2{iV`IbX+GZ%3^N7=!h z5R%pc>NOoznTC4+V%eix!)=>=>E=<4ER6J{7>xuHON1Een*a?>plmnbkeje z)MUePh)6^Y8N=97B6FYslPuQ%8~U-V6Xt|EEQsqOc1!HdrqrgzSV0SJ?wUCexxsCS zTg+1jm15kC)Q|U$qZtYsCJZkE+Pd{xl1aO&%^+p0VBVn^f4l0t9KDN2tL01v{)yrm zD;K-I9K7z+8t$=@x24;k0AzQexZ+}8Sf2DwW6=urpbf|^p;a-y^^y#?1NI`=e82d> zA_mmGidR7_yyz`Ny57+pZD;9;m`~sdcaRKQHuAxCAF)dM!PC~1#(1IDIAA*ERh0Z} z4?~b)+L~job(loUZlosU+~1k4Rqz2fU9`$TnR|hH;-4BML;MuXk?vSTOTeu<*9IqI zHNCO)j$M5CG2Cp=T*BGEY2QAr`8+hA?*Q5J2!f3ehJ{_f)#f$`G7^0W4pZ~{$q;U%7A9h$Gy3{HX}(Qev^z( zMWFKKgEhLxdhrU`ZwHV}RL57VFef%aR{k)b6pelCC#wZ=7BKdqdnLtS>jTFHK>9g1 z)0r=gX`uRb^Aq%tv8Lwe%ZOfcsU(Zn(h*Kzsc_B1^YkFXy8ey4vbK(+pfomdIsr^Q zn9dc7QiLpBdqK=nOVN+GuBV`E<-wCAto9nO;WvXyGA6%rlY z0vm(8yr3s7yzz#xKbePr7(-rQY-O`ut@?YYMP)fbT#oh$vIZpQyu#dcZaW}|uCGf6 z^#z2oQIz1!@=*#ZESawX4@DpD4O`cI=E1boCk zpsov(A~m|ctg2!P;6A=|S-rUqD&7Cp-knF~wDt`gzf+NfLYb8^L>fuPP=?ZksDv~R zng^B05K@McG)RPq5;8WBWXdd4hRlv3N|9tJ-n$OZdDeT*`#j4ztoNVqUh7`%weM^1 z>vw(jx_|BIy7scQUM(&f`5mfT^Q^;?CR+#DgzPuyJVvXl(o)GMQp3Dr4Es-b@W;>o zKg|7{B-haILi1O4ypZ|Ugzw4jKeXugYjXNgO5?6(4z2A|oOSv9(dOslGW={awW9(g z*SP8_T8wrc)BkPyuBJ;y6jfvozOOb@&vsis@S#xb*}7%@Di4M(%J2Enw0y{?#I#RQ%JpKZlE>v!+elvfoS_)=E@95D z;ONkqMw<^lY2dsd*Z%jSI|rVrj<*};JbJ*31@AUh z-H@HNA>I6D<;$82lSIU>kJ&eOQC8)u3kesN#98GonN*>r-f8U0h!<^j0)4+|hF%rZ z4IFmA+AIHQ(f+rO#E;Ilk~(j>DJmhY*Y$br#(U+gcT1@Z7N5|fsA=b9&9$>LJm0I0 zHrc*>bh%CGz4ZKH^${vj)0XY@)tRa@H~;0m=2h;~+Mf^Ju_-RU(9`a+`@W!vUB;tp ztg21!Ia;1u;W){*`CWknwn`xbpZ*HXdKDI6fJD zAR^8tXiK|Q#)+T zvFF7bKMXFGous%SL7}k!6nSO6^LMv=lI&>KDQnabW9gy+`{ahEwq0_n-{)C-?si`` zBYJzhdiCnL4&~Q}J1QB>+v{9C-CymIj;Lksh)ZVSd0(syRi>xv_1&nn((Q@iQc0iq zPJ5egIN?2DT>hA#b(6!M_THj@JNeiw?bNtYSLVN7TsvdO%dR>R3EdBA*Uw$Ja`%wT znOUutwUzbL8kTJpW}#IW&{adkYj}8WikL=Su4wY}Wj8+ElDQh)bk+qAopx#RM$=vT z95(e&St;5&y83~MaTonTgCwM%bzNF#W-;RGk&W?P!abLr?5$z0z4rRf>|wjkPgs-l z*zV=l=TU=?-f))JPm>NcH{U6t@0|ZSYV?_X{rV|wdy%ro{X_Hai=6_RUDOso@o>K0 zz8bC3XKh6^M;*U9bb|7}SJiRaeVdB(RMsyEA9*J>xNocS*}|@sck(nQZPq^B;jv3e zhsR!?uXeWW9(}+~cB!LW-Kp6vE0^A#D|zy%ytL$Io8lT6U`0+uuH#cgEE)Rn^h{tlz8l1;INkKlYcFb3c?cvq)v<%jr5*E-%Y& zXv|-`IW@P3PxYDumKv#(bJa3+5Dm#3bZPV6es?|H+!pAogCqG>u7~XYtiFsIdx?X079HrCygd5e|p4W9;Uxns)U@g^}%#UlBiUCUR)K?y;Q%Uzaa%+-jm) zv{~_e9@g*s6ld(tGqR2FxMMAD9c$5IP3y1E+xL=D&e|F~Y9HD~Jj4c|7ZaFa)_ z-_pJ3BR_7^3hO1?>$-){{B4PXTYF@jK3RwlGHsi!5mRiwa%TP2P8x^CpN=@~J5EHa z=K5hlsklZnS0h=MHf`!8W15NDw@sAHJhR-iOXbr#|6Z@BcNnmxIO}VHo!HoJe!Atx zYRy+gn7it!4cAR`Rh(8eQtFK8LRE{7`(t-SS9_N}_V?Ov)qab@qzNI~lO0U7RR?SM z?OENs_n=2#2N>*KbIjnyjQM7lJqL+-d(G-Nb>l{3iHl0x?-dMKd+S9`V6cq#DTQl& zV$8~3Sr@i4_;7M)@dbS=+dE?Gg5y8WeWN??(BUcD3Z6BQzI?psQNmE`eV?69UVAfQ z#HqR2HsfnG? zmg8NMU2EQ)c-~Fg@BWQDj-g*@XfBY+P2ji1^wIA=4yMb)(V(YyVo^L&!vNA zX-LXE;izU)bl*%A>1tWpS4r%wLTmNk=3d{>?uc@R{)Y-rXYn8#+XW&{RVx$U+AAJu zD>mEXuv%=CQR0qhkI-`$k0g62xN0odc2r-V5K!1>`gShpReEVv@l}Eh0f9~2B`O+tF0^2b-jLQ!spT!3X^3*)ej6$ zbyIr3sO&+rn`X<0ew?;PW^c6Ix!tatmhF=rnSEkMSpB?uKjUqYx8#Q^KfYUBk~d`N z9Cx=>p&O>}tzK1Uw`%`V_2vW9#Bb;CNK0>OWA-j=Lc85Px7HaM$zD>dIDhY|?6GF& zt4HeCTgSzI(d@RgdfiReVRkRJXG-i78-6;-r%&g=D-T6pRplK0RJbu-<<<&cNrBOX z*9R;Q9<*I@VBW(B>+YugCC}Al9bVfZ`{u6gV!vrF=#%GM{bHb>jIDUPM*~_JFFm^2 zSLBA9wOv}2LX&N~Rz)uh{P?7x#<4}((D+AFVi&DG?r50RHC1u4a?y+r!zwQ97+4*v zvB);{jC$0rz=7qLPJYrDy-_v7YV*k@!Ir*ha$&{$oB3F|SboZJwJMr0;@Y&>%BR}m zp9@}38{~3LL|-^Y;JFW`)?bZ=n>|0gJ;}oMOSX4XnRbuy9|oDsFAr}0u#K2Qm*CZ# zBb732P8+?H-+8s5WZIREuRo4`xG}He+FVWl-9w9R7@lv^bN%(rhaaj*RjGDLe>P{? zE%loEQjsAgS~c}OE;QAjKf%yx@q7O;;bCX{Nuvo67_Lv zI{gP3el++n{LA+AY@0gON8Jm&3tXdfE+-Xx3{O?Q8xiN?aM>$P!@k>QjaMsA`L({} zs1#@4&v?)Ma{JDncQ+p0dZTqj{FD@H#==FV~%C5g&9Mo9}sCCl%07`-yVR8@S1@@+qf3j0L8vF-Y5EFWLKaB!!sk#P#v zE>G03<|iIr59pvBeV0U z=`Q2W=I$GlS6Y|KTht8*H@Gumz^AUx#ikN@L!vS|bv|}bGw$MO=arjP)Q69p6jiiZ z{q>oW;&WN=)W)if$9 z*)geSWMEJquUqG*g(cT zf1YbhfU;+MZ@6MIH~35D(*7x%wn!;oi;{6Nmy z%&-wtX0}`#_wq~bv4p)+W=cv+OqK3R-m0?C=s7+~G`T*kNss8x5lxj3yqIMAQa`NK zi@c1Q)uT?#3!9WRRm(L~zqsu@L(^E(q=ol7tqFf{`kq6fOK+oA9n||LJ;_n&q&hWx zfkf)Mg-Omqi@Js7i#^*lQaDVq#3F}Gx%nNEFSNMYX>Mj*^syWtDfgT{DmS-wtJ=A) zb7|Puf|J1mGxMFl2wE;#7N=O(uGd|;3r2P0lT*Y#7PQYARAzg%dHjN;MCVD}5@l0n zcFJ?jU%kP?DB^v~ZBi@d?YvXg+*Phm(9yH&+Ybr4Uy&5p_0X972M%RPr2_fVcFlqk zqvOruBkL6gf=*}a% z)lCPtoUz2+xKy&hGpmi|if&ao4!-**-ko?cHf@TA)6~Ij9;djQZ?|h1)BpCB(-w}7 z?|c;3si+^;s+4v+Zg{_+si{NagAH#~9PAI@Fs(XKeL=<``tVqlwM|c7*1Er^c0+T& zBa!!1m!2|O(K_!+aoHG;A}y<@su9jJnw%>&f7-rPR#o2V$y=;wZ{N+WuSfDWdj%`;Lv{)! zlUmw$A0b=Ye!p1Hsv92nJmsqH^*v*}=|S^L4nbY^4~eqZ`Eq=-=cnm6W*_iG4~3_8n9r^aA7ZE?o+9Vo>s%N2wU=l3 zu28t!J~6zNzpVS`vbf~sU3yvDTOZY~wf4lzu%?F%mgwth z)|8bj_bl;n(3?Hka`S`Ev&xLtO^tLv7%|^*-EQv<8?th}R*t%V=YBwsjy4JJdw3K$ ztX%h-{lmpauT9<|>pkUlZG}cc_OyJh+toMnzOHeoJMH_hGD%Zi*WFpwOr%4zTMC`u zh20K+eKm83n^Nn-i8Bpa7oXc}f6$=Ei8+7%`Txa|zdizNsnbyO-~d+zfuN<1h{T>w zl9`<{D+J>HegZXbKNmGSA3FzECp9~V0JlIVPhYiBjWZ?_O`QDw?OdEj&31GPF!eT{ z5r`NHq=a!aZpGX!#3|TM(0-O(pq<+9<=wp8og5A9{9R4#dVUv6OpH0?t3mE0+XE#UB z&8x90;fqe12*icE!r9HwKfut-(J6Smpp}PR!+c|Fgex>G@b+>U`>zs)0vRFF_Z`)^ zJ%3|?n1`ExK!8BbxS{dic1R08|mf4U3fZTE+l@xNA)>SI1*ylsS{ za0uC8pP+`36dQ{E-+S|W)4pw6L`XeeAZG6t;BP2s>fz+&65uM_P7@K4V4wdo1^=kJ z@QHeVU%`BV#J@GM;YBy!r|ym4w)obFhLSCVnlx$h^8*n+_4?z1h>HFEK!i`G{dgc^ z5;nO2O9*DT~&ksa+hxf+=X{zw^0}pD3VQnMJU26H9sCo3$b4j%HPSaqEZsSB9yN9R!W%4vh~$1UFQ{ga85vAbR~q*mUAkum8!Kk$~{?Y zod!vB+`!8``=5$vXX8G|I&a0f)IOjO8 zbKc;*$$883qLrmuvpwvLfY!g>04B1r=O*?t~14j*7zc&b)_jTwU=LobY$L% zxYjn0aLtr{>af8848Q;kzyJ)u01TX^0XL#|Bl8>|9Dkag<#z$cf&mzS0T_S*7=Qs7 zfB_hQ0T_UR(=~wc|LOY0SBC)@fB_hQ0T_S*7=Qs7fB_hQfoU+{#{b;^pV=}erThPF ztGr2SdpAif>?Wxl-6XY^o1`{zlhpEUlG?XTQY*Ge>Z&$L{m~|=)7d2TESse6W0TZJ zY?3;HO;T^KN$S!yN&U7aH;*X)*!0Qj&Aps|K0yKf6ZRW|dl-NL7=Qs7fB_hQ0T_S* z7=VH4FyIEvJpP~gZcb}{vMJP)O`R4}JY`x)@l0v@6ozloCTgqJk8WJ}0fM+NC39~+n&PJ52f7iV8?nZwX+@!+w*CcI{yw$- z0QdR}^vszv6}vmZk|58N-ZvTY=FWX`dg)BgnWE5kJsv+jtp60LX%7boKmY;|fB*y_ z009U<00Izz00jO!0(HA0T?mlJ_w9M^ufw?SAOHaf zKmY;|fB*y_009U<00J{jpqA(R3pUKBsc#mU)HdWqpAO#b=mK`Vr zAOHafKmY;|fB*y_009U<00Of{pq?lASJTKEA61u+jWd4xA6tI)(d<8U7sr5di&z-CGWMN)4M9(e)8UY;q(r~yPCYWH;c*_8!*S+o-U%>)w{g&4qBP>{mJy`U*V~hn0vd2HX?igjk z!tBXsuRF$But3}R;&sPJ3l?h6zB=8d-ia*MhKH=3%3mD~0uX=z1Rwwb2tZ&81m^P0 zKF`o^*dX{9bqb#br-T3mAOHafKmY;|fB*y_009V`EP;8}0ZtY$?+yV7KmY;|fB*y_ z009U<00I!0Ap(uO4q$ZwwJjZ7tP4mf^s15$uPUkSs*4Ikzgiq@Z25AgUbsL$ zMfyNELI45~fB*y_009U<00Izzz)TTn=83bd1I&Nd;Vn;AVm(=DwbNxK)lQd{QafE% zLhW=}>9o^jCDTrql}bBZRwC_mS!whnJ2Uvsl*_1nXG;4x_&jO<2A?PG+u-x0{Th6p zv`>T2llEuudD6ZNK2P3{LGKp-diRffhm>7Fv`xIo$6{PYi9!GZ5P$##AOHafKtKun E21+W84gdfE literal 0 HcmV?d00001