From 93f2fe0c35a59ea3611cf96d60f6c3f341b76645 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Thu, 18 Feb 2010 21:38:53 +0100 Subject: [PATCH 01/15] updated to 0.4 multiverse --- akka-util-java/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-util-java/pom.xml b/akka-util-java/pom.xml index e0a729491b..3daa008792 100644 --- a/akka-util-java/pom.xml +++ b/akka-util-java/pom.xml @@ -27,7 +27,7 @@ org.multiverse multiverse-alpha - 0.3 + 0.4-SNAPSHOT jar-with-dependencies From 87f66d0c4f3666a0f497cda23f9e2a1a85cb5340 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Mon, 22 Feb 2010 13:22:10 +0100 Subject: [PATCH 02/15] upgraded to multiverse 0.4-SNAPSHOT --- akka-core/pom.xml | 28 ++ akka-core/src/main/scala/actor/Actor.scala | 8 +- .../src/main/scala/stm/Transaction.scala | 132 ++++--- .../scala/stm/TransactionManagement.scala | 4 +- .../main/scala/stm/TransactionalState.scala | 5 +- akka-util-java/pom.xml | 28 -- .../akka/stm/AtomicTemplate.java | 341 ------------------ pom.xml | 52 ++- 8 files changed, 136 insertions(+), 462 deletions(-) delete mode 100644 akka-util-java/src/main/java/se/scalablesolutions/akka/stm/AtomicTemplate.java diff --git a/akka-core/pom.xml b/akka-core/pom.xml index d6ca57ebfe..5d58430aa6 100644 --- a/akka-core/pom.xml +++ b/akka-core/pom.xml @@ -46,6 +46,34 @@ netty 3.2.0.ALPHA3 + + org.multiverse + multiverse-alpha + 0.4-SNAPSHOT + jar-with-dependencies + + + org.multiverse + multiverse-core + + + asm + asm-tree + + + asm + asm-analysis + + + asm + asm-commons + + + asm + asm-util + + + org.scala-tools javautils diff --git a/akka-core/src/main/scala/actor/Actor.scala b/akka-core/src/main/scala/actor/Actor.scala index 41b2673bfa..a6d9231b10 100644 --- a/akka-core/src/main/scala/actor/Actor.scala +++ b/akka-core/src/main/scala/actor/Actor.scala @@ -911,10 +911,10 @@ trait Actor extends TransactionManagement { try { if (isTransactionRequiresNew && !isTransactionInScope) { - if (senderFuture.isEmpty) throw new StmException( - "Can't continue transaction in a one-way fire-forget message send" + - "\n\tE.g. using Actor '!' method or Active Object 'void' method" + - "\n\tPlease use the Actor '!!' method or Active Object method with non-void return type") + //if (senderFuture.isEmpty) throw new StmException( + // "Can't continue transaction in a one-way fire-forget message send" + + // "\n\tE.g. using Actor '!' method or Active Object 'void' method" + + // "\n\tPlease use the Actor '!!' method or Active Object method with non-void return type") atomic { proceed } diff --git a/akka-core/src/main/scala/stm/Transaction.scala b/akka-core/src/main/scala/stm/Transaction.scala index 1637b4c906..d535f98433 100644 --- a/akka-core/src/main/scala/stm/Transaction.scala +++ b/akka-core/src/main/scala/stm/Transaction.scala @@ -13,9 +13,10 @@ import se.scalablesolutions.akka.util.Logging import org.multiverse.api.{Transaction => MultiverseTransaction} import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance import org.multiverse.api.ThreadLocalTransaction._ -import org.multiverse.templates.OrElseTemplate +import org.multiverse.commitbarriers.VetoCommitBarrier import scala.collection.mutable.HashMap +import org.multiverse.templates.{TransactionTemplate, OrElseTemplate} class NoTransactionInScopeException extends RuntimeException class TransactionRetryException(message: String) extends RuntimeException(message) @@ -30,8 +31,8 @@ class TransactionRetryException(message: String) extends RuntimeException(messag * Here are some examples (assuming implicit transaction family name in scope): *
  * import se.scalablesolutions.akka.stm.Transaction._
- * 
- * atomic {
+ *
+ * atomic  {
  *   .. // do something within a transaction
  * }
  * 
@@ -39,8 +40,8 @@ class TransactionRetryException(message: String) extends RuntimeException(messag * Example of atomic transaction management using atomic block with retry count: *
  * import se.scalablesolutions.akka.stm.Transaction._
- * 
- * atomic(maxNrOfRetries) {
+ *
+ * atomic(maxNrOfRetries)  {
  *   .. // do something within a transaction
  * }
  * 
@@ -49,10 +50,10 @@ class TransactionRetryException(message: String) extends RuntimeException(messag * Which is a good way to reduce contention and transaction collisions. *
  * import se.scalablesolutions.akka.stm.Transaction._
- * 
- * atomically {
+ *
+ * atomically  {
  *   .. // try to do something
- * } orElse {
+ * } orElse  {
  *   .. // if transaction clashes try do do something else to minimize contention
  * }
  * 
@@ -61,11 +62,11 @@ class TransactionRetryException(message: String) extends RuntimeException(messag * *
  * import se.scalablesolutions.akka.stm.Transaction._
- * for (tx <- Transaction) {
+ * for (tx <- Transaction)  {
  *   ... // do transactional stuff
  * }
  *
- * val result = for (tx <- Transaction) yield {
+ * val result = for (tx <- Transaction) yield  {
  *   ... // do transactional stuff yielding a result
  * }
  * 
@@ -78,17 +79,17 @@ class TransactionRetryException(message: String) extends RuntimeException(messag * * // You can use them together with Transaction in a for comprehension since * // TransactionalRef is also monadic - * for { + * for { * tx <- Transaction * ref <- refs * } { * ... // use the ref inside a transaction * } * - * val result = for { + * val result = for { * tx <- Transaction * ref <- refs - * } yield { + * } yield { * ... // use the ref inside a transaction, yield a result * } * @@ -101,57 +102,61 @@ object Transaction extends TransactionManagement { /** * See ScalaDoc on class. */ - def map[T](f: Transaction => T)(implicit transactionFamilyName: String): T = atomic { f(getTransactionInScope) } + def map[T](f: Transaction => T)(implicit transactionFamilyName: String): T = atomic {f(getTransactionInScope)} /** * See ScalaDoc on class. */ - def flatMap[T](f: Transaction => T)(implicit transactionFamilyName: String): T = atomic { f(getTransactionInScope) } + def flatMap[T](f: Transaction => T)(implicit transactionFamilyName: String): T = atomic {f(getTransactionInScope)} /** * See ScalaDoc on class. */ - def foreach(f: Transaction => Unit)(implicit transactionFamilyName: String): Unit = atomic { f(getTransactionInScope) } + def foreach(f: Transaction => Unit)(implicit transactionFamilyName: String): Unit = atomic {f(getTransactionInScope)} /** * Creates a "pure" STM atomic transaction and by-passes all transactions hooks * such as persistence etc. * Only for internal usage. */ - private[akka] def pureAtomic[T](body: => T): T = new AtomicTemplate[T]( - getGlobalStmInstance, "internal", false, false, TransactionManagement.MAX_NR_OF_RETRIES) { + private[akka] def pureAtomic[T](body: => T): T = new TransactionTemplate[T]() { def execute(mtx: MultiverseTransaction): T = body }.execute() /** * See ScalaDoc on class. */ - def atomic[T](body: => T)(implicit transactionFamilyName: String): T = new AtomicTemplate[T]( - getGlobalStmInstance, transactionFamilyName, false, false, TransactionManagement.MAX_NR_OF_RETRIES) { - def execute(mtx: MultiverseTransaction): T = body - override def postStart(mtx: MultiverseTransaction) = { - val tx = new Transaction - tx.transaction = Some(mtx) - setTransaction(Some(tx)) - } - override def postCommit = { - if (isTransactionInScope) getTransactionInScope.commit - else throw new IllegalStateException("No transaction in scope") - } - }.execute() + def atomic[T](body: => T)(implicit transactionFamilyName: String): T = { + new TransactionTemplate[T]() { // FIXME take factory + def execute(mtx: MultiverseTransaction): T = body + + override def onStart(mtx: MultiverseTransaction) = { + val tx = new Transaction + tx.transaction = Some(mtx) + setTransaction(Some(tx)) + } + + override def onPostCommit = { + if (isTransactionInScope) getTransactionInScope.commit + else throw new IllegalStateException("No transaction in scope") + } + }.execute() + } /** * See ScalaDoc on class. */ def atomic[T](retryCount: Int)(body: => T)(implicit transactionFamilyName: String): T = { - new AtomicTemplate[T](getGlobalStmInstance, transactionFamilyName, false, false, retryCount) { + new TransactionTemplate[T]() { // FIXME take factory def execute(mtx: MultiverseTransaction): T = body - override def postStart(mtx: MultiverseTransaction) = { + + override def onStart(mtx: MultiverseTransaction) = { val tx = new Transaction tx.transaction = Some(mtx) setTransaction(Some(tx)) } - override def postCommit = { + + override def onPostCommit = { if (isTransactionInScope) getTransactionInScope.commit else throw new IllegalStateException("No transaction in scope") } @@ -162,14 +167,16 @@ object Transaction extends TransactionManagement { * See ScalaDoc on class. */ def atomicReadOnly[T](retryCount: Int)(body: => T)(implicit transactionFamilyName: String): T = { - new AtomicTemplate[T](getGlobalStmInstance, transactionFamilyName, false, true, retryCount) { + new TransactionTemplate[T]() { // FIXME take factory def execute(mtx: MultiverseTransaction): T = body - override def postStart(mtx: MultiverseTransaction) = { + + override def onStart(mtx: MultiverseTransaction) = { val tx = new Transaction tx.transaction = Some(mtx) setTransaction(Some(tx)) } - override def postCommit = { + + override def onPostCommit = { if (isTransactionInScope) getTransactionInScope.commit else throw new IllegalStateException("No transaction in scope") } @@ -180,14 +187,16 @@ object Transaction extends TransactionManagement { * See ScalaDoc on class. */ def atomicReadOnly[T](body: => T): T = { - new AtomicTemplate[T](true) { + new TransactionTemplate[T]() { // FIXME take factory def execute(mtx: MultiverseTransaction): T = body - override def postStart(mtx: MultiverseTransaction) = { + + override def onStart(mtx: MultiverseTransaction) = { val tx = new Transaction tx.transaction = Some(mtx) setTransaction(Some(tx)) } - override def postCommit = { + + override def onPostCommit = { if (isTransactionInScope) getTransactionInScope.commit else throw new IllegalStateException("No transaction in scope") } @@ -206,6 +215,7 @@ object Transaction extends TransactionManagement { def elseBody[A](firstBody: => A) = new { def orElse(secondBody: => A) = new OrElseTemplate[A] { def run(t: MultiverseTransaction) = firstBody + def orelserun(t: MultiverseTransaction) = secondBody }.execute() } @@ -216,30 +226,38 @@ object Transaction extends TransactionManagement { */ @serializable class Transaction extends Logging { import Transaction._ - + val id = Transaction.idFactory.incrementAndGet @volatile private[this] var status: TransactionStatus = TransactionStatus.New private[akka] var transaction: Option[MultiverseTransaction] = None private[this] val persistentStateMap = new HashMap[String, Committable] private[akka] val depth = new AtomicInteger(0) - + //private[akka] val transactionSet = new VetoCommitBarrier + + //transactionSet.vetoCommit(this) + // --- public methods --------- + def abort = synchronized { + // transactionSet.abort + } + def commit = synchronized { pureAtomic { persistentStateMap.values.foreach(_.commit) TransactionManagement.clearTransaction } + //transactionSet.vetoCommit(this) status = TransactionStatus.Completed } - def isNew = synchronized { status == TransactionStatus.New } + def isNew = synchronized {status == TransactionStatus.New} - def isActive = synchronized { status == TransactionStatus.Active } + def isActive = synchronized {status == TransactionStatus.Active} - def isCompleted = synchronized { status == TransactionStatus.Completed } + def isCompleted = synchronized {status == TransactionStatus.Completed} - def isAborted = synchronized { status == TransactionStatus.Aborted } + def isAborted = synchronized {status == TransactionStatus.Aborted} // --- internal methods --------- @@ -259,13 +277,13 @@ object Transaction extends TransactionManagement { private def ensureIsActiveOrAborted = if (!(status == TransactionStatus.Active || status == TransactionStatus.Aborted)) - throw new IllegalStateException( - "Expected ACTIVE or ABORTED transaction - current status [" + status + "]: " + toString) + throw new IllegalStateException( + "Expected ACTIVE or ABORTED transaction - current status [" + status + "]: " + toString) private def ensureIsActiveOrNew = if (!(status == TransactionStatus.Active || status == TransactionStatus.New)) - throw new IllegalStateException( - "Expected ACTIVE or NEW transaction - current status [" + status + "]: " + toString) + throw new IllegalStateException( + "Expected ACTIVE or NEW transaction - current status [" + status + "]: " + toString) // For reinitialize transaction after sending it over the wire private[akka] def reinit = synchronized { @@ -277,14 +295,14 @@ object Transaction extends TransactionManagement { } override def equals(that: Any): Boolean = synchronized { - that != null && - that.isInstanceOf[Transaction] && - that.asInstanceOf[Transaction].id == this.id + that != null && + that.isInstanceOf[Transaction] && + that.asInstanceOf[Transaction].id == this.id } - - override def hashCode(): Int = synchronized { id.toInt } - - override def toString(): String = synchronized { "Transaction[" + id + ", " + status + "]" } + + override def hashCode(): Int = synchronized {id.toInt} + + override def toString(): String = synchronized {"Transaction[" + id + ", " + status + "]"} } /** diff --git a/akka-core/src/main/scala/stm/TransactionManagement.scala b/akka-core/src/main/scala/stm/TransactionManagement.scala index 2dd7ed9c79..1f2ede3024 100644 --- a/akka-core/src/main/scala/stm/TransactionManagement.scala +++ b/akka-core/src/main/scala/stm/TransactionManagement.scala @@ -26,7 +26,7 @@ object TransactionManagement extends TransactionManagement { def isTransactionalityEnabled = TRANSACTION_ENABLED.get def disableTransactions = TRANSACTION_ENABLED.set(false) - private[akka] val currentTransaction: ThreadLocal[Option[Transaction]] = new ThreadLocal[Option[Transaction]]() { + private[akka] val currentTransaction = new ThreadLocal[Option[Transaction]]() { override protected def initialValue: Option[Transaction] = None } } @@ -52,6 +52,8 @@ trait TransactionManagement extends Logging { private[akka] def isTransactionInScope = currentTransaction.get.isDefined + private[akka] def isTransactionTopLevel = if (isTransactionInScope) getTransactionInScope.isTopLevel + private[akka] def incrementTransaction = if (isTransactionInScope) getTransactionInScope.increment private[akka] def decrementTransaction = if (isTransactionInScope) getTransactionInScope.decrement diff --git a/akka-core/src/main/scala/stm/TransactionalState.scala b/akka-core/src/main/scala/stm/TransactionalState.scala index 6003a89f89..195b134271 100644 --- a/akka-core/src/main/scala/stm/TransactionalState.scala +++ b/akka-core/src/main/scala/stm/TransactionalState.scala @@ -8,8 +8,7 @@ import se.scalablesolutions.akka.stm.Transaction.atomic import se.scalablesolutions.akka.stm.NoTransactionInScopeException import se.scalablesolutions.akka.collection._ import se.scalablesolutions.akka.util.UUID - -import org.multiverse.datastructures.refs.manual.Ref; +import org.multiverse.stms.alpha.AlphaRef /** * Example Scala usage: @@ -78,7 +77,7 @@ class TransactionalRef[T] extends Transactional { implicit val txInitName = "TransactionalRef:Init" val uuid = UUID.newUuid.toString - private[this] val ref: Ref[T] = atomic { new Ref } + private[this] val ref: AlphaRef[T] = atomic { new AlphaRef } def swap(elem: T) = { ensureIsInTransaction diff --git a/akka-util-java/pom.xml b/akka-util-java/pom.xml index 3daa008792..6db055b223 100644 --- a/akka-util-java/pom.xml +++ b/akka-util-java/pom.xml @@ -24,34 +24,6 @@ protobuf-java 2.2.0
- - org.multiverse - multiverse-alpha - 0.4-SNAPSHOT - jar-with-dependencies - - - org.multiverse - multiverse-core - - - asm - asm-tree - - - asm - asm-analysis - - - asm - asm-commons - - - asm - asm-util - - - diff --git a/akka-util-java/src/main/java/se/scalablesolutions/akka/stm/AtomicTemplate.java b/akka-util-java/src/main/java/se/scalablesolutions/akka/stm/AtomicTemplate.java deleted file mode 100644 index a693ff1248..0000000000 --- a/akka-util-java/src/main/java/se/scalablesolutions/akka/stm/AtomicTemplate.java +++ /dev/null @@ -1,341 +0,0 @@ -package se.scalablesolutions.akka.stm; - -import static org.multiverse.api.GlobalStmInstance.getGlobalStmInstance; -import org.multiverse.api.Stm; -import static org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction; -import static org.multiverse.api.ThreadLocalTransaction.setThreadLocalTransaction; -import org.multiverse.api.Transaction; -import org.multiverse.api.TransactionStatus; -import org.multiverse.api.exceptions.CommitFailureException; -import org.multiverse.api.exceptions.LoadException; -import org.multiverse.api.exceptions.RetryError; -import org.multiverse.api.exceptions.TooManyRetriesException; -import org.multiverse.templates.AbortedException; -import org.multiverse.utils.latches.CheapLatch; -import org.multiverse.utils.latches.Latch; - -import static java.lang.String.format; -import java.util.logging.Logger; - -/** - * A Template that handles the boilerplate code for transactions. A transaction will be placed if none is available - * around a section and if all goes right, commits at the end. - *

- * example: - *

- * new AtomicTemplate(){
- *    Object execute(Transaction t){
- *        queue.push(1);
- *        return null;
- *    }
- * }.execute();
- * 
- *

- * It could also be that the transaction is retried (e.g. caused by optimistic locking failures). This is also a task - * for template. In the future this retry behavior will be customizable. - *

- * If a transaction already is available on the TransactionThreadLocal, no new transaction is started and essentially - * the whole AtomicTemplate is ignored. - *

- * If no transaction is available on the TransactionThreadLocal, a new one will be created and used during the execution - * of the AtomicTemplate and will be removed once the AtomicTemplate finishes. - *

- * All uncaught throwable's lead to a rollback of the transaction. - *

- * AtomicTemplates are not thread-safe to use. - *

- * AtomicTemplates can completely work without threadlocals. See the {@link AtomicTemplate#AtomicTemplate(org.multiverse.api.Stm - * ,String, boolean, boolean, int)} for more information. - * - * @author Peter Veentjer - */ -public abstract class AtomicTemplate { - - private final static Logger logger = Logger.getLogger(AtomicTemplate.class.getName()); - - private final Stm stm; - private final boolean ignoreThreadLocalTransaction; - private final int retryCount; - private final boolean readonly; - private int attemptCount; - private final String familyName; - - /** - * Creates a new AtomicTemplate that uses the STM stored in the GlobalStm and works the the {@link - * org.multiverse.utils.ThreadLocalTransaction}. - */ - public AtomicTemplate() { - this(getGlobalStmInstance()); - } - - public AtomicTemplate(boolean readonly) { - this(getGlobalStmInstance(), null, false, readonly, Integer.MAX_VALUE); - } - - /** - * Creates a new AtomicTemplate using the provided stm. The transaction used is stores/retrieved from the {@link - * org.multiverse.utils.ThreadLocalTransaction}. - * - * @param stm the stm to use for transactions. - * @throws NullPointerException if stm is null. - */ - public AtomicTemplate(Stm stm) { - this(stm, null, false, false, Integer.MAX_VALUE); - } - - public AtomicTemplate(String familyName, boolean readonly, int retryCount) { - this(getGlobalStmInstance(), familyName, false, readonly, retryCount); - } - - /** - * Creates a new AtomicTemplate that uses the provided STM. This method is provided to make Multiverse easy to - * integrate with environment that don't want to depend on threadlocals. - * - * @param stm the stm to use for transactions. - * @param ignoreThreadLocalTransaction true if this Template should completely ignore the ThreadLocalTransaction. - * This is useful for using the AtomicTemplate in other environments that don't - * want to depend on threadlocals but do want to use the AtomicTemplate. - * @throws NullPointerException if stm is null. - */ - public AtomicTemplate(Stm stm, String familyName, boolean ignoreThreadLocalTransaction, boolean readonly, - int retryCount) { - if (stm == null) { - throw new NullPointerException(); - } - if (retryCount < 0) { - throw new IllegalArgumentException(); - } - this.stm = stm; - this.ignoreThreadLocalTransaction = ignoreThreadLocalTransaction; - this.readonly = readonly; - this.retryCount = retryCount; - this.familyName = familyName; - } - - public String getFamilyName() { - return familyName; - } - - /** - * Returns the current attempt. Value will always be larger than zero and increases everytime the transaction needs - * to be retried. - * - * @return the current attempt count. - */ - public final int getAttemptCount() { - return attemptCount; - } - - /** - * Returns the number of retries that this AtomicTemplate is allowed to do. The returned value will always be equal - * or larger than 0. - * - * @return the number of retries. - */ - public final int getRetryCount() { - return retryCount; - } - - /** - * Returns the {@link Stm} used by this AtomicTemplate to execute transactions on. - * - * @return the Stm used by this AtomicTemplate. - */ - public final Stm getStm() { - return stm; - } - - /** - * Check if this AtomicTemplate ignores the ThreadLocalTransaction. - * - * @return true if this AtomicTemplate ignores the ThreadLocalTransaction, false otherwise. - */ - public final boolean isIgnoreThreadLocalTransaction() { - return ignoreThreadLocalTransaction; - } - - /** - * Checks if this AtomicTemplate executes readonly transactions. - * - * @return true if it executes readonly transactions, false otherwise. - */ - public final boolean isReadonly() { - return readonly; - } - - /** - * This is the method can be overridden to do pre-start tasks. - */ - public void preStart() { - } - - /** - * This is the method can be overridden to do post-start tasks. - * - * @param t the transaction used for this execution. - */ - public void postStart(Transaction t) { - } - - /** - * This is the method can be overridden to do pre-commit tasks. - */ - public void preCommit() { - } - - /** - * This is the method can be overridden to do post-commit tasks. - */ - public void postCommit() { - } - - /** - * This is the method that needs to be implemented. - * - * @param t the transaction used for this execution. - * @return the result of the execution. - * - * @throws Exception the Exception thrown - */ - public abstract E execute(Transaction t) throws Exception; - - /** - * Executes the template. - * - * @return the result of the {@link #execute(org.multiverse.api.Transaction)} method. - * - * @throws InvisibleCheckedException if a checked exception was thrown while executing the {@link - * #execute(org.multiverse.api.Transaction)} method. - * @throws AbortedException if the exception was explicitly aborted. - * @throws TooManyRetriesException if the template retried the transaction too many times. The cause of the last - * failure (also an exception) is included as cause. So you have some idea where - * to look for problems - */ - public final E execute() { - try { - return executeChecked(); - } catch (Exception ex) { - if (ex instanceof RuntimeException) { - throw (RuntimeException) ex; - } else { - throw new AtomicTemplate.InvisibleCheckedException(ex); - } - } - } - - /** - * Executes the Template and rethrows the checked exception instead of wrapping it in a InvisibleCheckedException. - * - * @return the result - * - * @throws Exception the Exception thrown inside the {@link #execute(org.multiverse.api.Transaction)} - * method. - * @throws AbortedException if the exception was explicitly aborted. - * @throws TooManyRetriesException if the template retried the transaction too many times. The cause of the last - * failure (also an exception) is included as cause. So you have some idea where to - * look for problems - */ - public final E executeChecked() throws Exception { - preStart(); - Transaction t = getTransaction(); - if (noUsableTransaction(t)) { - t = startTransaction(); - setTransaction(t); - postStart(t); - try { - attemptCount = 1; - Exception lastRetryCause = null; - while (attemptCount - 1 <= retryCount) { - boolean abort = true; - boolean reset = false; - try { - E result = execute(t); - if (t.getStatus().equals(TransactionStatus.aborted)) { - String msg = format("Transaction with familyname %s is aborted", t.getFamilyName()); - throw new AbortedException(msg); - } - preCommit(); - t.commit(); - abort = false; - reset = false; - postCommit(); - return result; - } catch (RetryError e) { - Latch latch = new CheapLatch(); - t.abortAndRegisterRetryLatch(latch); - latch.awaitUninterruptible(); - //since the abort is already done, no need to do it again. - abort = false; - } catch (CommitFailureException ex) { - lastRetryCause = ex; - reset = true; - //ignore, just retry the transaction - } catch (LoadException ex) { - lastRetryCause = ex; - reset = true; - //ignore, just retry the transaction - } finally { - if (abort) { - t.abort(); - if (reset) { - t = t.abortAndReturnRestarted(); - setTransaction(t); - } - } - } - attemptCount++; - } - - throw new TooManyRetriesException("Too many retries", lastRetryCause); - } finally { - setTransaction(null); - } - } else { - return execute(t); - } - } - - private Transaction startTransaction() { - return readonly ? stm.startReadOnlyTransaction(familyName) : stm.startUpdateTransaction(familyName); - } - - private boolean noUsableTransaction(Transaction t) { - return t == null || t.getStatus() != TransactionStatus.active; - } - - /** - * Gets the current Transaction stored in the TransactionThreadLocal. - *

- * If the ignoreThreadLocalTransaction is set, the threadlocal stuff is completeley ignored. - * - * @return the found transaction, or null if none is found. - */ - private Transaction getTransaction() { - return ignoreThreadLocalTransaction ? null : getThreadLocalTransaction(); - } - - /** - * Stores the transaction in the TransactionThreadLocal. - *

- * This call is ignored if the ignoreThreadLocalTransaction is true. - * - * @param t the transaction to set (is allowed to be null). - */ - private void setTransaction(Transaction t) { - if (!ignoreThreadLocalTransaction) { - setThreadLocalTransaction(t); - } - } - - public static class InvisibleCheckedException extends RuntimeException { - - public InvisibleCheckedException(Exception cause) { - super(cause); - } - - @Override - public Exception getCause() { - return (Exception) super.getCause(); - } - } -} diff --git a/pom.xml b/pom.xml index 3cfc2839a8..d2885abf61 100644 --- a/pom.xml +++ b/pom.xml @@ -14,25 +14,27 @@ Akka implements a unique hybrid of: - * Actors , which gives you: - * Simple and high-level abstractions for concurrency and parallelism. - * Asynchronous, non-blocking and highly performant event-driven programming model. - * Very lightweight event-driven processes (create ~6.5 million actors on 4 G RAM). - * Supervision hierarchies with let-it-crash semantics. For writing highly fault-tolerant systems that never stop, systems that self-heal. - * Software Transactional Memory (STM). (Distributed transactions coming soon). - * Transactors: combine actors and STM into transactional actors. Allows you to compose atomic message flows with automatic rollback and retry. - * Remoting: highly performant distributed actors with remote supervision and error management. - * Cluster membership management. + * Actors , which gives you: + * Simple and high-level abstractions for concurrency and parallelism. + * Asynchronous, non-blocking and highly performant event-driven programming model. + * Very lightweight event-driven processes (create ~6.5 million actors on 4 G RAM). + * Supervision hierarchies with let-it-crash semantics. For writing highly fault-tolerant systems that never stop, + systems that self-heal. + * Software Transactional Memory (STM). (Distributed transactions coming soon). + * Transactors: combine actors and STM into transactional actors. Allows you to compose atomic message flows with + automatic rollback and retry. + * Remoting: highly performant distributed actors with remote supervision and error management. + * Cluster membership management. Akka also has a set of add-on modules: - * Persistence: A set of pluggable back-end storage modules that works in sync with the STM. - * Cassandra distributed and highly scalable database. - * MongoDB document database. - * Redis data structures database (upcoming) - * REST (JAX-RS): Expose actors as REST services. - * Comet: Expose actors as Comet services. - * Security: Digest and Kerberos based security. - * Microkernel: Run Akka as a stand-alone kernel. + * Persistence: A set of pluggable back-end storage modules that works in sync with the STM. + * Cassandra distributed and highly scalable database. + * MongoDB document database. + * Redis data structures database (upcoming) + * REST (JAX-RS): Expose actors as REST services. + * Comet: Expose actors as Comet services. + * Security: Digest and Kerberos based security. + * Microkernel: Run Akka as a stand-alone kernel. @@ -155,15 +157,9 @@ http://www.lag.net/repo - multiverse-releases - http://multiverse.googlecode.com/svn/maven-repository/releases - - false - - - - multiverse-snaphosts - http://multiverse.googlecode.com/svn/maven-repository/snapshots + repository.codehaus.org + Codehaus Maven Repository + http://repository.codehaus.org maven2-repository.dev.java.net @@ -258,7 +254,7 @@ src/main/scala src/test/scala - + org.apache.maven.plugins maven-enforcer-plugin 1.0-beta-1 @@ -276,7 +272,7 @@ - ${env.AKKA_HOME}/embedded-repo + ${env.AKKA_HOME}/embedded-repo From 28c6cc7b54e79e6813b5691733b34219fc8fd3e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Tue, 23 Feb 2010 11:15:37 +0100 Subject: [PATCH 03/15] renamed actor api --- akka-core/src/main/scala/actor/Actor.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/akka-core/src/main/scala/actor/Actor.scala b/akka-core/src/main/scala/actor/Actor.scala index a6d9231b10..d6185b65a3 100644 --- a/akka-core/src/main/scala/actor/Actor.scala +++ b/akka-core/src/main/scala/actor/Actor.scala @@ -98,9 +98,7 @@ object Actor extends Logging { * The actor is started when created. * Example: *

-   * import Actor._
-   *
-   * val a = actor  {
+   * val a = Actor.init  {
    *   ... // init stuff
    * } receive  {
    *   case msg => ... // handle message
@@ -108,7 +106,7 @@ object Actor extends Logging {
    * 
* */ - def actor[A](body: => Unit) = { + def init[A](body: => Unit) = { def handler[A](body: => Unit) = new { def receive(handler: PartialFunction[Any, Unit]) = new Actor() { start From 21c53c2ffda8a1002712e2923e78c73eb959f9fb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Tue, 23 Feb 2010 19:49:01 +0100 Subject: [PATCH 04/15] Upgraded to Multiverse 0.4 and its 2PC CommitBarriers, all tests pass --- akka-core/pom.xml | 23 --- akka-core/src/main/scala/actor/Actor.scala | 81 ++++++--- .../src/main/scala/dispatch/Reactor.scala | 9 +- .../src/main/scala/stm/Transaction.scala | 158 ++++++++---------- .../scala/stm/TransactionManagement.scala | 75 ++++++--- .../main/scala/stm/TransactionalState.scala | 2 +- .../src/test/scala/InMemoryActorTest.scala | 28 ++-- akka-core/src/test/scala/ShutdownSpec.scala | 5 +- .../akka/api/InMemNestedStateTest.java | 3 +- akka-patterns/src/main/scala/Agent.scala | 24 +-- .../src/test/scala/ActorPatternsTest.scala | 6 +- akka-patterns/src/test/scala/AgentTest.scala | 27 +-- .../src/main/scala/Storage.scala | 22 +-- akka.iml | 13 +- config/akka-reference.conf | 6 +- pom.xml | 11 +- 16 files changed, 259 insertions(+), 234 deletions(-) diff --git a/akka-core/pom.xml b/akka-core/pom.xml index 5d58430aa6..d2116cd0f1 100644 --- a/akka-core/pom.xml +++ b/akka-core/pom.xml @@ -50,29 +50,6 @@ org.multiverse multiverse-alpha 0.4-SNAPSHOT - jar-with-dependencies - - - org.multiverse - multiverse-core - - - asm - asm-tree - - - asm - asm-analysis - - - asm - asm-commons - - - asm - asm-util - - org.scala-tools diff --git a/akka-core/src/main/scala/actor/Actor.scala b/akka-core/src/main/scala/actor/Actor.scala index d6185b65a3..2919cc5c71 100644 --- a/akka-core/src/main/scala/actor/Actor.scala +++ b/akka-core/src/main/scala/actor/Actor.scala @@ -10,7 +10,7 @@ import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, F import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.stm.Transaction._ import se.scalablesolutions.akka.stm.TransactionManagement._ -import se.scalablesolutions.akka.stm.{StmException, TransactionManagement} +import se.scalablesolutions.akka.stm.TransactionManagement import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.RemoteRequest import se.scalablesolutions.akka.remote.{RemoteProtocolBuilder, RemoteClient, RemoteRequestIdFactory} import se.scalablesolutions.akka.serialization.Serializer @@ -21,6 +21,7 @@ import org.multiverse.api.ThreadLocalTransaction._ import java.util.{Queue, HashSet} import java.util.concurrent.ConcurrentLinkedQueue import java.net.InetSocketAddress +import org.multiverse.commitbarriers.CountDownCommitBarrier /** * Implements the Transactor abstraction. E.g. a transactional actor. @@ -72,7 +73,7 @@ object Actor extends Logging { val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost") val PORT = config.getInt("akka.remote.server.port", 9999) - object Sender{ + object Sender { implicit val Self: Option[Actor] = None } @@ -193,7 +194,7 @@ object Actor extends Logging { */ trait Actor extends TransactionManagement { implicit protected val self: Option[Actor] = Some(this) - implicit protected val transactionFamily: String = this.getClass.getName + implicit protected val transactionFamilyName: String = this.getClass.getName // Only mutable for RemoteServer in order to maintain identity across nodes private[akka] var _uuid = UUID.newUuid.toString @@ -216,6 +217,7 @@ trait Actor extends TransactionManagement { private[akka] var _replyToAddress: Option[InetSocketAddress] = None private[akka] val _mailbox: Queue[MessageInvocation] = new ConcurrentLinkedQueue[MessageInvocation] + // ==================================== // protected fields // ==================================== @@ -333,8 +335,8 @@ trait Actor extends TransactionManagement { /** * User overridable callback/setting. * - * Partial function implementing the server logic. - * To be implemented by subclassing server. + * Partial function implementing the actor logic. + * To be implemented by subclassing actor. *

* Example code: *

@@ -782,6 +784,11 @@ trait Actor extends TransactionManagement {
   }
 
   protected[akka] def postMessageToMailbox(message: Any, sender: Option[Actor]): Unit = {
+    if (isTransactionSetInScope) {
+      log.trace("Adding transaction for %s with message [%s] to transaction set", toString, message)
+      getTransactionSetInScope.incParties
+    }
+
     if (_remoteAddress.isDefined) {
       val requestBuilder = RemoteRequest.newBuilder
           .setId(RemoteRequestIdFactory.nextId)
@@ -793,8 +800,7 @@ trait Actor extends TransactionManagement {
           .setIsEscaped(false)
       
       val id = registerSupervisorAsRemoteActor
-      if (id.isDefined)
-        requestBuilder.setSupervisorUuid(id.get)
+      if (id.isDefined) requestBuilder.setSupervisorUuid(id.get)
 
       // set the source fields used to reply back to the original sender
       // (i.e. not the remote proxy actor)
@@ -813,18 +819,24 @@ trait Actor extends TransactionManagement {
       RemoteProtocolBuilder.setMessage(message, requestBuilder)
       RemoteClient.clientFor(_remoteAddress.get).send(requestBuilder.build, None)
     } else {
-      val invocation = new MessageInvocation(this, message, None, sender, currentTransaction.get)
+      val invocation = new MessageInvocation(this, message, None, sender, transactionSet.get)
       if (_isEventBased) {
         _mailbox.add(invocation)
         if (_isSuspended) invocation.send
       } else invocation.send
     }
+    clearTransactionSet
   }
 
   protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(
       message: Any, 
       timeout: Long,
       senderFuture: Option[CompletableFutureResult]): CompletableFutureResult = {
+    if (isTransactionSetInScope) {
+      log.trace("Adding transaction for %s with message [%s] to transaction set", toString, message)    
+      getTransactionSetInScope.incParties
+    }
+    
     if (_remoteAddress.isDefined) {
       val requestBuilder = RemoteRequest.newBuilder
           .setId(RemoteRequestIdFactory.nextId)
@@ -838,16 +850,18 @@ trait Actor extends TransactionManagement {
       val id = registerSupervisorAsRemoteActor
       if (id.isDefined) requestBuilder.setSupervisorUuid(id.get)
       val future = RemoteClient.clientFor(_remoteAddress.get).send(requestBuilder.build, senderFuture)
+      clearTransactionSet
       if (future.isDefined) future.get
       else throw new IllegalStateException("Expected a future from remote call to actor " + toString)
     } else {
       val future = if (senderFuture.isDefined) senderFuture.get
                    else new DefaultCompletableFutureResult(timeout)
-      val invocation = new MessageInvocation(this, message, Some(future), None, currentTransaction.get)
+      val invocation = new MessageInvocation(this, message, Some(future), None, transactionSet.get)
       if (_isEventBased) {
         _mailbox.add(invocation)
         invocation.send
       } else invocation.send
+      clearTransactionSet
       future
     }
   }
@@ -856,6 +870,7 @@ trait Actor extends TransactionManagement {
    * Callback for the dispatcher. E.g. single entry point to the user code and all protected[this] methods.
    */
   private[akka] def invoke(messageHandle: MessageInvocation) = synchronized {
+    log.trace("%s is invoked with message %s", toString, messageHandle)
     try {
       if (TransactionManagement.isTransactionalityEnabled) transactionalDispatch(messageHandle)
       else dispatch(messageHandle)
@@ -867,7 +882,7 @@ trait Actor extends TransactionManagement {
   }
 
   private def dispatch[T](messageHandle: MessageInvocation) = {
-    setTransaction(messageHandle.tx)
+    setTransactionSet(messageHandle.transactionSet)
 
     val message = messageHandle.message //serializeMessage(messageHandle.message)
     senderFuture = messageHandle.future
@@ -889,43 +904,55 @@ trait Actor extends TransactionManagement {
   }
 
   private def transactionalDispatch[T](messageHandle: MessageInvocation) = {
-    setTransaction(messageHandle.tx)
+    var topLevelTransaction = false
+    val txSet: Option[CountDownCommitBarrier] =
+      if (messageHandle.transactionSet.isDefined) messageHandle.transactionSet
+      else {
+        topLevelTransaction = true // FIXME create a new internal atomic block that can wait for X seconds if top level tx
+        if (isTransactionRequiresNew) {
+          log.trace("Creating a new transaction set (top-level transaction) \nfor actor %s \nwith message %s", toString, messageHandle)
+          Some(createNewTransactionSet)
+        } else None
+      }
+    setTransactionSet(txSet)
 
     val message = messageHandle.message //serializeMessage(messageHandle.message)
     senderFuture = messageHandle.future
     sender = messageHandle.sender
 
+    def clearTx = {
+      clearTransactionSet
+      clearTransaction
+    }
+
     def proceed = {
-      try {
-        incrementTransaction
-        if (base.isDefinedAt(message)) base(message) // invoke user actor's receive partial function
-        else throw new IllegalArgumentException(
-          "Actor " + toString + " could not process message [" + message + "]" +
-           "\n\tsince no matching 'case' clause in its 'receive' method could be found")
-      } finally {
-        decrementTransaction
-      }
+      if (base.isDefinedAt(message)) base(message) // invoke user actor's receive partial function
+      else throw new IllegalArgumentException(
+        toString + " could not process message [" + message + "]" +
+        "\n\tsince no matching 'case' clause in its 'receive' method could be found")
+      setTransactionSet(txSet) // restore transaction set to allow atomic block to do commit
     }
 
     try {
-      if (isTransactionRequiresNew && !isTransactionInScope) {
-        //if (senderFuture.isEmpty) throw new StmException(
-        //  "Can't continue transaction in a one-way fire-forget message send" +
-        //  "\n\tE.g. using Actor '!' method or Active Object 'void' method" +
-        //  "\n\tPlease use the Actor '!!' method or Active Object method with non-void return type")
+      if (isTransactionRequiresNew) {
         atomic {
           proceed
         }
       } else proceed
     } catch {
+      case e: IllegalStateException => {}
       case e =>
+        // abort transaction set
+        if (isTransactionSetInScope) try { getTransactionSetInScope.abort } catch { case e: IllegalStateException => {} }
         Actor.log.error(e, "Exception when invoking \n\tactor [%s] \n\twith message [%s]", this, message)
+
         if (senderFuture.isDefined) senderFuture.get.completeWithException(this, e)
-        clearTransaction // need to clear currentTransaction before call to supervisor
+        clearTx  // need to clear currentTransaction before call to supervisor
+
         // FIXME to fix supervisor restart of remote actor for oneway calls, inject a supervisor proxy that can send notification back to client
         if (_supervisor.isDefined) _supervisor.get ! Exit(this, e)
     } finally {
-      clearTransaction
+      clearTx
     }
   }
 
diff --git a/akka-core/src/main/scala/dispatch/Reactor.scala b/akka-core/src/main/scala/dispatch/Reactor.scala
index f7bfa52215..b5d4d634f6 100644
--- a/akka-core/src/main/scala/dispatch/Reactor.scala
+++ b/akka-core/src/main/scala/dispatch/Reactor.scala
@@ -7,16 +7,17 @@ package se.scalablesolutions.akka.dispatch
 import java.util.List
 
 import se.scalablesolutions.akka.util.{HashCode, Logging}
-import se.scalablesolutions.akka.stm.Transaction
 import se.scalablesolutions.akka.actor.Actor
 
 import java.util.concurrent.ConcurrentHashMap
 
+import org.multiverse.commitbarriers.CountDownCommitBarrier
+
 final class MessageInvocation(val receiver: Actor,
                               val message: Any,
                               val future: Option[CompletableFutureResult],
                               val sender: Option[Actor],
-                              val tx: Option[Transaction]) {
+                              val transactionSet: Option[CountDownCommitBarrier]) {
   if (receiver eq null) throw new IllegalArgumentException("receiver is null")
 
   def invoke = receiver.invoke(this)
@@ -37,13 +38,13 @@ final class MessageInvocation(val receiver: Actor,
     that.asInstanceOf[MessageInvocation].message == message
   }
 
-  override def toString(): String = synchronized {
+  override def toString = synchronized {
     "MessageInvocation[" +
      "\n\tmessage = " + message +
      "\n\treceiver = " + receiver +
      "\n\tsender = " + sender +
      "\n\tfuture = " + future +
-     "\n\ttx = " + tx +
+     "\n\ttransactionSet = " + transactionSet +
      "\n]"
   }
 }
diff --git a/akka-core/src/main/scala/stm/Transaction.scala b/akka-core/src/main/scala/stm/Transaction.scala
index d535f98433..133c292a6f 100644
--- a/akka-core/src/main/scala/stm/Transaction.scala
+++ b/akka-core/src/main/scala/stm/Transaction.scala
@@ -7,16 +7,18 @@ package se.scalablesolutions.akka.stm
 import java.util.concurrent.atomic.AtomicLong
 import java.util.concurrent.atomic.AtomicInteger
 
+import scala.collection.mutable.HashMap
+
 import se.scalablesolutions.akka.state.Committable
 import se.scalablesolutions.akka.util.Logging
 
 import org.multiverse.api.{Transaction => MultiverseTransaction}
 import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance
 import org.multiverse.api.ThreadLocalTransaction._
-import org.multiverse.commitbarriers.VetoCommitBarrier
-
-import scala.collection.mutable.HashMap
 import org.multiverse.templates.{TransactionTemplate, OrElseTemplate}
+import org.multiverse.utils.backoff.ExponentialBackoffPolicy
+import org.multiverse.stms.alpha.AlphaStm
+import java.util.concurrent.TimeUnit
 
 class NoTransactionInScopeException extends RuntimeException
 class TransactionRetryException(message: String) extends RuntimeException(message)
@@ -98,21 +100,42 @@ class TransactionRetryException(message: String) extends RuntimeException(messag
  */
 object Transaction extends TransactionManagement {
   val idFactory = new AtomicLong(-1L)
+/*
+  import AlphaStm._
+  private val defaultTxBuilder = new AlphaTransactionFactoryBuilder
+  defaultTxBuilder.setReadonly(false)
+  defaultTxBuilder.setInterruptible(INTERRUPTIBLE)
+  defaultTxBuilder.setMaxRetryCount(MAX_NR_OF_RETRIES)
+  defaultTxBuilder.setPreventWriteSkew(PREVENT_WRITE_SKEW)
+  defaultTxBuilder.setAutomaticReadTracking(AUTOMATIC_READ_TRACKING)
+  defaultTxBuilder.setSmartTxLengthSelector(SMART_TX_LENGTH_SELECTOR)
+  defaultTxBuilder.setBackoffPolicy(new ExponentialBackoffPolicy)
+  private val readOnlyTxBuilder = new AlphaStm.AlphaTransactionFactoryBuilder
+  readOnlyTxBuilder.setReadonly(true)
+  readOnlyTxBuilder.setInterruptible(INTERRUPTIBLE)
+  readOnlyTxBuilder.setMaxRetryCount(MAX_NR_OF_RETRIES)
+  readOnlyTxBuilder.setPreventWriteSkew(PREVENT_WRITE_SKEW)
+  readOnlyTxBuilder.setAutomaticReadTracking(AUTOMATIC_READ_TRACKING)
+  readOnlyTxBuilder.setSmartTxLengthSelector(SMART_TX_LENGTH_SELECTOR)
+  readOnlyTxBuilder.setBackoffPolicy(new ExponentialBackoffPolicy)
+  */
+  /**
+   * See ScalaDoc on class.
+   */
+  def map[T](f: => T)(implicit transactionFamilyName: String): T =
+    atomic {f}
 
   /**
    * See ScalaDoc on class.
    */
-  def map[T](f: Transaction => T)(implicit transactionFamilyName: String): T = atomic {f(getTransactionInScope)}
+  def flatMap[T](f: => T)(implicit transactionFamilyName: String): T =
+    atomic {f}
 
   /**
    * See ScalaDoc on class.
    */
-  def flatMap[T](f: Transaction => T)(implicit transactionFamilyName: String): T = atomic {f(getTransactionInScope)}
-
-  /**
-   * See ScalaDoc on class.
-   */
-  def foreach(f: Transaction => Unit)(implicit transactionFamilyName: String): Unit = atomic {f(getTransactionInScope)}
+  def foreach(f: => Unit)(implicit transactionFamilyName: String): Unit =
+    atomic {f}
 
   /**
    * Creates a "pure" STM atomic transaction and by-passes all transactions hooks
@@ -127,82 +150,39 @@ object Transaction extends TransactionManagement {
    * See ScalaDoc on class.
    */
   def atomic[T](body: => T)(implicit transactionFamilyName: String): T = {
+    //    defaultTxBuilder.setFamilyName(transactionFamilyName)
+    //    new TransactionTemplate[T](defaultTxBuilder.build) {
     new TransactionTemplate[T]() { // FIXME take factory
-      def execute(mtx: MultiverseTransaction): T = body
+      def execute(mtx: MultiverseTransaction): T = {
+        val result = body
+
+        log.trace("Committing transaction [%s] \nwith family name [%s] \nby joining transaction set")
+        getTransactionSetInScope.joinCommit(mtx)
+
+        // FIXME tryJoinCommit(mtx, TransactionManagement.TRANSACTION_TIMEOUT, TimeUnit.MILLISECONDS) 
+        //getTransactionSetInScope.tryJoinCommit(mtx, TransactionManagement.TRANSACTION_TIMEOUT, TimeUnit.MILLISECONDS)
+
+        clearTransaction
+        result
+      }
 
       override def onStart(mtx: MultiverseTransaction) = {
+        val txSet = if (!isTransactionSetInScope) createNewTransactionSet
+                    else getTransactionSetInScope
         val tx = new Transaction
         tx.transaction = Some(mtx)
         setTransaction(Some(tx))
-      }
 
-      override def onPostCommit = {
-        if (isTransactionInScope) getTransactionInScope.commit
-        else throw new IllegalStateException("No transaction in scope")
+        txSet.registerOnCommitTask(new Runnable() {
+          def run = tx.commit
+        })
+        txSet.registerOnAbortTask(new Runnable() {
+          def run = tx.abort
+        })
       }
     }.execute()
   }
 
-  /**
-   * See ScalaDoc on class.
-   */
-  def atomic[T](retryCount: Int)(body: => T)(implicit transactionFamilyName: String): T = {
-    new TransactionTemplate[T]() { // FIXME take factory
-      def execute(mtx: MultiverseTransaction): T = body
-
-      override def onStart(mtx: MultiverseTransaction) = {
-        val tx = new Transaction
-        tx.transaction = Some(mtx)
-        setTransaction(Some(tx))
-      }
-
-      override def onPostCommit = {
-        if (isTransactionInScope) getTransactionInScope.commit
-        else throw new IllegalStateException("No transaction in scope")
-      }
-    }.execute
-  }
-
-  /**
-   * See ScalaDoc on class.
-   */
-  def atomicReadOnly[T](retryCount: Int)(body: => T)(implicit transactionFamilyName: String): T = {
-    new TransactionTemplate[T]() { // FIXME take factory
-      def execute(mtx: MultiverseTransaction): T = body
-
-      override def onStart(mtx: MultiverseTransaction) = {
-        val tx = new Transaction
-        tx.transaction = Some(mtx)
-        setTransaction(Some(tx))
-      }
-
-      override def onPostCommit = {
-        if (isTransactionInScope) getTransactionInScope.commit
-        else throw new IllegalStateException("No transaction in scope")
-      }
-    }.execute
-  }
-
-  /**
-   * See ScalaDoc on class.
-   */
-  def atomicReadOnly[T](body: => T): T = {
-    new TransactionTemplate[T]() { // FIXME take factory
-      def execute(mtx: MultiverseTransaction): T = body
-
-      override def onStart(mtx: MultiverseTransaction) = {
-        val tx = new Transaction
-        tx.transaction = Some(mtx)
-        setTransaction(Some(tx))
-      }
-
-      override def onPostCommit = {
-        if (isTransactionInScope) getTransactionInScope.commit
-        else throw new IllegalStateException("No transaction in scope")
-      }
-    }.execute
-  }
-
   /**
    * See ScalaDoc on class.
    */
@@ -215,7 +195,6 @@ object Transaction extends TransactionManagement {
   def elseBody[A](firstBody: => A) = new {
     def orElse(secondBody: => A) = new OrElseTemplate[A] {
       def run(t: MultiverseTransaction) = firstBody
-
       def orelserun(t: MultiverseTransaction) = secondBody
     }.execute()
   }
@@ -227,37 +206,34 @@ object Transaction extends TransactionManagement {
 @serializable class Transaction extends Logging {
   import Transaction._
 
+  log.trace("Creating %s", toString)
   val id = Transaction.idFactory.incrementAndGet
   @volatile private[this] var status: TransactionStatus = TransactionStatus.New
   private[akka] var transaction: Option[MultiverseTransaction] = None
   private[this] val persistentStateMap = new HashMap[String, Committable]
   private[akka] val depth = new AtomicInteger(0)
-  //private[akka] val transactionSet = new VetoCommitBarrier
-
-  //transactionSet.vetoCommit(this)
 
   // --- public methods ---------
 
-  def abort = synchronized {
-  //  transactionSet.abort
-  }
-
   def commit = synchronized {
+    log.trace("Committing transaction %s", toString)    
     pureAtomic {
       persistentStateMap.values.foreach(_.commit)
-      TransactionManagement.clearTransaction
     }
-    //transactionSet.vetoCommit(this)
     status = TransactionStatus.Completed
   }
 
-  def isNew = synchronized {status == TransactionStatus.New}
+  def abort = synchronized {
+    log.trace("Aborting transaction %s", toString)    
+  }
 
-  def isActive = synchronized {status == TransactionStatus.Active}
+  def isNew = synchronized { status == TransactionStatus.New }
 
-  def isCompleted = synchronized {status == TransactionStatus.Completed}
+  def isActive = synchronized { status == TransactionStatus.Active }
 
-  def isAborted = synchronized {status == TransactionStatus.Aborted}
+  def isCompleted = synchronized { status == TransactionStatus.Completed }
+
+  def isAborted = synchronized { status == TransactionStatus.Aborted }
 
   // --- internal methods ---------
 
@@ -300,9 +276,9 @@ object Transaction extends TransactionManagement {
         that.asInstanceOf[Transaction].id == this.id
   }
 
-  override def hashCode(): Int = synchronized {id.toInt}
+  override def hashCode(): Int = synchronized { id.toInt }
 
-  override def toString(): String = synchronized {"Transaction[" + id + ", " + status + "]"}
+  override def toString = synchronized { "Transaction[" + id + ", " + status + "]" }
 }
 
 /**
diff --git a/akka-core/src/main/scala/stm/TransactionManagement.scala b/akka-core/src/main/scala/stm/TransactionManagement.scala
index 1f2ede3024..60a6ae6de3 100644
--- a/akka-core/src/main/scala/stm/TransactionManagement.scala
+++ b/akka-core/src/main/scala/stm/TransactionManagement.scala
@@ -9,53 +9,80 @@ import java.util.concurrent.atomic.AtomicBoolean
 import se.scalablesolutions.akka.util.Logging
 
 import org.multiverse.api.ThreadLocalTransaction._
+import org.multiverse.commitbarriers.CountDownCommitBarrier
 
 class StmException(msg: String) extends RuntimeException(msg)
 
-class TransactionAwareWrapperException(
-    val cause: Throwable, val tx: Option[Transaction]) extends RuntimeException(cause) {
-  override def toString(): String = "TransactionAwareWrapperException[" + cause + ", " + tx + "]"
+class TransactionAwareWrapperException(val cause: Throwable, val tx: Option[Transaction]) extends RuntimeException(cause) {
+  override def toString = "TransactionAwareWrapperException[" + cause + ", " + tx + "]"
 }
 
 object TransactionManagement extends TransactionManagement {
   import se.scalablesolutions.akka.Config._
 
-  val MAX_NR_OF_RETRIES = config.getInt("akka.stm.max-nr-of-retries", 100)
-  val TRANSACTION_ENABLED = new AtomicBoolean(config.getBool("akka.stm.service", false))
-
+  val TRANSACTION_ENABLED =      new AtomicBoolean(config.getBool("akka.stm.service", false))
+  val FAIR_TRANSACTIONS =        config.getBool("akka.stm.fair", true)
+  val INTERRUPTIBLE =            config.getBool("akka.stm.interruptible", true)
+  val MAX_NR_OF_RETRIES =        config.getInt("akka.stm.max-nr-of-retries", 1000)
+  val TRANSACTION_TIMEOUT =      config.getInt("akka.stm.timeout", 10000)
+  val SMART_TX_LENGTH_SELECTOR = config.getBool("akka.stm.smart-tx-length-selector", true)
   def isTransactionalityEnabled = TRANSACTION_ENABLED.get
+
   def disableTransactions = TRANSACTION_ENABLED.set(false)
 
-  private[akka] val currentTransaction = new ThreadLocal[Option[Transaction]]() {
+  private[akka] val transactionSet = new ThreadLocal[Option[CountDownCommitBarrier]]() {
+    override protected def initialValue: Option[CountDownCommitBarrier] = None
+  }
+
+  private[akka] val transaction = new ThreadLocal[Option[Transaction]]() {
     override protected def initialValue: Option[Transaction] = None
   }
+
+  private[akka] def getTransactionSet: CountDownCommitBarrier = {
+    val option = transactionSet.get
+    if ((option eq null) || option.isEmpty) throw new IllegalStateException("No TransactionSet in scope")
+    option.get
+  }
+
+  private[akka] def getTransaction: Transaction = {
+    val option = transaction.get
+    if ((option eq null) || option.isEmpty) throw new IllegalStateException("No Transaction in scope")
+    option.get
+  }
 }
 
 trait TransactionManagement extends Logging {
-  import TransactionManagement.currentTransaction
 
-  private[akka] def createNewTransaction = currentTransaction.set(Some(new Transaction))
-
-  private[akka] def setTransaction(transaction: Option[Transaction]) = if (transaction.isDefined) {
-    val tx = transaction.get
-    currentTransaction.set(transaction)
-    if (tx.transaction.isDefined) setThreadLocalTransaction(tx.transaction.get)
-    else throw new IllegalStateException("No transaction defined")
+  private[akka] def createNewTransactionSet: CountDownCommitBarrier = {
+    val txSet = new CountDownCommitBarrier(1, TransactionManagement.FAIR_TRANSACTIONS)
+    TransactionManagement.transactionSet.set(Some(txSet))
+    txSet
   }
 
+  private[akka] def setTransactionSet(txSet: Option[CountDownCommitBarrier]) =
+    if (txSet.isDefined) TransactionManagement.transactionSet.set(txSet)
+
+  private[akka] def setTransaction(tx: Option[Transaction]) =
+    if (tx.isDefined) TransactionManagement.transaction.set(tx)
+
+  private[akka] def clearTransactionSet = TransactionManagement.transactionSet.set(None)
+
   private[akka] def clearTransaction = {
-    currentTransaction.set(None)
+    TransactionManagement.transaction.set(None)
     setThreadLocalTransaction(null)
   }
 
-  private[akka] def getTransactionInScope = currentTransaction.get.get
-  
-  private[akka] def isTransactionInScope = currentTransaction.get.isDefined
+  private[akka] def getTransactionSetInScope = TransactionManagement.getTransactionSet
 
-  private[akka] def isTransactionTopLevel = if (isTransactionInScope) getTransactionInScope.isTopLevel
+  private[akka] def getTransactionInScope = TransactionManagement.getTransaction
 
-  private[akka] def incrementTransaction = if (isTransactionInScope) getTransactionInScope.increment
-
-  private[akka] def decrementTransaction = if (isTransactionInScope) getTransactionInScope.decrement
-}
+  private[akka] def isTransactionSetInScope = {
+    val option = TransactionManagement.transactionSet.get
+    (option ne null) && option.isDefined
+  }
 
+  private[akka] def isTransactionInScope = {
+    val option = TransactionManagement.transaction.get
+    (option ne null) && option.isDefined
+  }
+}
\ No newline at end of file
diff --git a/akka-core/src/main/scala/stm/TransactionalState.scala b/akka-core/src/main/scala/stm/TransactionalState.scala
index 195b134271..1b52faf969 100644
--- a/akka-core/src/main/scala/stm/TransactionalState.scala
+++ b/akka-core/src/main/scala/stm/TransactionalState.scala
@@ -77,7 +77,7 @@ class TransactionalRef[T] extends Transactional {
   implicit val txInitName = "TransactionalRef:Init"
   val uuid = UUID.newUuid.toString
 
-  private[this] val ref: AlphaRef[T] = atomic { new AlphaRef }
+  private[this] lazy val ref: AlphaRef[T] = new AlphaRef
 
   def swap(elem: T) = {
     ensureIsInTransaction
diff --git a/akka-core/src/test/scala/InMemoryActorTest.scala b/akka-core/src/test/scala/InMemoryActorTest.scala
index cd06b80d0a..d4be98fcaa 100644
--- a/akka-core/src/test/scala/InMemoryActorTest.scala
+++ b/akka-core/src/test/scala/InMemoryActorTest.scala
@@ -23,7 +23,7 @@ case class SuccessOneWay(key: String, value: String)
 case class FailureOneWay(key: String, value: String, failer: Actor)
 
 class InMemStatefulActor extends Actor {
-  timeout = 100000
+  timeout = 5000
   makeTransactionRequired
 
   private lazy val mapState = TransactionalState.newMap[String, String]
@@ -86,8 +86,8 @@ class InMemFailerActor extends Actor {
 }
                                                         
 class InMemoryActorTest extends JUnitSuite {
+  import Actor.Sender.Self
 
-  /*
   @Test
   def shouldOneWayMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
     val stateful = new InMemStatefulActor
@@ -98,7 +98,7 @@ class InMemoryActorTest extends JUnitSuite {
     Thread.sleep(1000)
     assert("new state" === (stateful !! GetMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess")).get)
   }
-  */
+
   @Test
   def shouldMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
     val stateful = new InMemStatefulActor
@@ -107,7 +107,7 @@ class InMemoryActorTest extends JUnitSuite {
     stateful !! Success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state") // transactionrequired
     assert("new state" === (stateful !! GetMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess")).get)
   }
-  /*
+
   @Test
   def shouldOneWayMapShouldRollbackStateForStatefulServerInCaseOfFailure = {
     val stateful = new InMemStatefulActor
@@ -120,7 +120,7 @@ class InMemoryActorTest extends JUnitSuite {
     Thread.sleep(1000)
     assert("init" === (stateful !! GetMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure")).get) // check that state is == init state
   }
-  */
+
   @Test
   def shouldMapShouldRollbackStateForStatefulServerInCaseOfFailure = {
     val stateful = new InMemStatefulActor
@@ -134,7 +134,7 @@ class InMemoryActorTest extends JUnitSuite {
     } catch {case e: RuntimeException => {}}
     assert("init" === (stateful !! GetMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure")).get) // check that state is == init state
   }
-  /*
+
   @Test
   def shouldOneWayVectorShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
     val stateful = new InMemStatefulActor
@@ -145,7 +145,7 @@ class InMemoryActorTest extends JUnitSuite {
     Thread.sleep(1000)
     assert(2 === (stateful !! GetVectorSize).get)
   }
-  */
+
   @Test
   def shouldVectorShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
     val stateful = new InMemStatefulActor
@@ -154,7 +154,7 @@ class InMemoryActorTest extends JUnitSuite {
     stateful !! Success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state") // transactionrequired
     assert(2 === (stateful !! GetVectorSize).get)
   }
-  /*
+
   @Test
   def shouldOneWayVectorShouldRollbackStateForStatefulServerInCaseOfFailure = {
     val stateful = new InMemStatefulActor
@@ -167,7 +167,7 @@ class InMemoryActorTest extends JUnitSuite {
     Thread.sleep(1000)
     assert(1 === (stateful !! GetVectorSize).get)
   }
-  */
+
   @Test
   def shouldVectorShouldRollbackStateForStatefulServerInCaseOfFailure = {
     val stateful = new InMemStatefulActor
@@ -181,7 +181,7 @@ class InMemoryActorTest extends JUnitSuite {
     } catch {case e: RuntimeException => {}}
     assert(1 === (stateful !! GetVectorSize).get)
   }
-  /*
+
   @Test
   def shouldOneWayRefShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
     val stateful = new InMemStatefulActor
@@ -192,7 +192,7 @@ class InMemoryActorTest extends JUnitSuite {
     Thread.sleep(1000)
     assert("new state" === (stateful !! GetRefState).get)
   }
-  */
+
   @Test
   def shouldRefShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
     val stateful = new InMemStatefulActor
@@ -201,7 +201,7 @@ class InMemoryActorTest extends JUnitSuite {
     stateful !! Success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state") // transactionrequired
     assert("new state" === (stateful !! GetRefState).get)
   }
-  /*
+
   @Test
   def shouldOneWayRefShouldRollbackStateForStatefulServerInCaseOfFailure = {
     val stateful = new InMemStatefulActor
@@ -212,9 +212,9 @@ class InMemoryActorTest extends JUnitSuite {
     failer.start
     stateful ! FailureOneWay("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", failer) // call failing transactionrequired method
     Thread.sleep(1000)
-    assert("init" === (stateful !! GetRefState).get) // check that state is == init state
+    assert("init" === (stateful !! (GetRefState, 1000000)).get) // check that state is == init state
   }
-  */
+
   @Test
   def shouldRefShouldRollbackStateForStatefulServerInCaseOfFailure = {
     val stateful = new InMemStatefulActor
diff --git a/akka-core/src/test/scala/ShutdownSpec.scala b/akka-core/src/test/scala/ShutdownSpec.scala
index ba03fbe902..20927bbfb1 100644
--- a/akka-core/src/test/scala/ShutdownSpec.scala
+++ b/akka-core/src/test/scala/ShutdownSpec.scala
@@ -2,9 +2,8 @@ package se.scalablesolutions.akka.remote
 
 import se.scalablesolutions.akka.actor.Actor
 
-object ActorShutdownSpec {
+object ActorShutdownRunner {
   def main(args: Array[String]) {
-
     class MyActor extends Actor {
       def receive = {
         case "test" => println("received test")
@@ -22,7 +21,7 @@ object ActorShutdownSpec {
 
 // case 2
 
-object RemoteServerAndClusterShutdownSpec {
+object RemoteServerAndClusterShutdownRunner {
   def main(args: Array[String]) {
     val s1 = new RemoteServer
     val s2 = new RemoteServer
diff --git a/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java b/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java
index 8a51feed6b..366403ef46 100644
--- a/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java
+++ b/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java
@@ -11,7 +11,7 @@ import static se.scalablesolutions.akka.config.JavaConfig.*;
 import se.scalablesolutions.akka.actor.*;
 import se.scalablesolutions.akka.Kernel;
 import junit.framework.TestCase;
-/*
+
 public class InMemNestedStateTest extends TestCase {
   static String messageLog = "";
 
@@ -133,4 +133,3 @@ public class InMemNestedStateTest extends TestCase {
     assertEquals("init", nested.getRefState()); // check that state is == init state
   }
 }
-*/
\ No newline at end of file
diff --git a/akka-patterns/src/main/scala/Agent.scala b/akka-patterns/src/main/scala/Agent.scala
index 4dd8640c32..aea74530a3 100644
--- a/akka-patterns/src/main/scala/Agent.scala
+++ b/akka-patterns/src/main/scala/Agent.scala
@@ -50,30 +50,30 @@ sealed class Agent[T] private (initialValue: T) extends Actor {
   * Periodically handles incoming messages
   */
   def receive = {
-    case FunctionHolder(fun: (T => T)) => atomic { updateData(fun(value.getOrWait)) }
+    case FunctionHolder(fun: (T => T)) => updateData(fun(value.getOrWait))
 
     case ValueHolder(x: T) => updateData(x)
 
-    case ProcedureHolder(fun: (T => Unit)) => atomic { fun(copyStrategy(value.getOrWait)) }
+    case ProcedureHolder(fun: (T => Unit)) => fun(copyStrategy(value.getOrWait))
   }
 
   /**
    * Specifies how a copy of the value is made, defaults to using identity
    */
-  protected def copyStrategy(t : T) : T = t
+  protected def copyStrategy(t: T): T = t
 
 
   /**
   * Updates the internal state with the value provided as a by-name parameter
   */
-  private final def updateData(newData: => T) : Unit = atomic { value.swap(newData) }
+  private final def updateData(newData: => T): Unit = value.swap(newData)
 
   /**
   * Submits a request to read the internal state.
   * A copy of the internal state will be returned, depending on the underlying effective copyStrategy.
   * Internally leverages the asynchronous getValue() method and then waits for its result on a CountDownLatch.
   */
-  final def get : T = {
+  final def get: T = {
     val ref = new AtomicReference[T]
     val latch = new CountDownLatch(1)
     get((x: T) => {ref.set(x); latch.countDown})
@@ -85,14 +85,14 @@ sealed class Agent[T] private (initialValue: T) extends Actor {
   * Asynchronously submits a request to read the internal state. The supplied function will be executed on the returned internal state value.
   * A copy of the internal state will be used, depending on the underlying effective copyStrategy.
   */
-  final def get(message: (T => Unit)) : Unit = this ! ProcedureHolder(message)
+  final def get(message: (T => Unit)): Unit = this ! ProcedureHolder(message)
 
   /**
   * Submits a request to read the internal state.
   * A copy of the internal state will be returned, depending on the underlying effective copyStrategy.
   * Internally leverages the asynchronous getValue() method and then waits for its result on a CountDownLatch.
   */
-  final def apply() : T = get
+  final def apply(): T = get
 
   /**
   * Asynchronously submits a request to read the internal state. The supplied function will be executed on the returned internal state value.
@@ -103,22 +103,22 @@ sealed class Agent[T] private (initialValue: T) extends Actor {
   /**
   * Submits the provided function for execution against the internal agent's state
   */
-  final def apply(message: (T => T)) : Unit = this ! FunctionHolder(message)
+  final def apply(message: (T => T)): Unit = this ! FunctionHolder(message)
 
   /**
   * Submits a new value to be set as the new agent's internal state
   */
-  final def apply(message: T) : Unit = this ! ValueHolder(message)
+  final def apply(message: T): Unit = this ! ValueHolder(message)
 
   /**
   * Submits the provided function for execution against the internal agent's state
   */
-  final def update(message: (T => T)) : Unit = this ! FunctionHolder(message)
+  final def update(message: (T => T)): Unit = this ! FunctionHolder(message)
 
   /**
   * Submits a new value to be set as the new agent's internal state
   */
-  final def update(message: T) : Unit = this ! ValueHolder(message)
+  final def update(message: T): Unit = this ! ValueHolder(message)
 }
 
 /**
@@ -135,7 +135,7 @@ object Agent {
   /**
    * Creates a new Agent of type T with the initial value of value
    */
-  def apply[T](value:T) : Agent[T] = new Agent(value)
+  def apply[T](value:T): Agent[T] = new Agent(value)
 
   /**
    * Creates a new Agent of type T with the initial value of value and with the specified copy function
diff --git a/akka-patterns/src/test/scala/ActorPatternsTest.scala b/akka-patterns/src/test/scala/ActorPatternsTest.scala
index 11f2664640..ae6ae5c0e8 100644
--- a/akka-patterns/src/test/scala/ActorPatternsTest.scala
+++ b/akka-patterns/src/test/scala/ActorPatternsTest.scala
@@ -21,12 +21,12 @@ class ActorPatternsTest extends junit.framework.TestCase with Suite with MustMat
       val (testMsg1,testMsg2,testMsg3,testMsg4) = ("test1","test2","test3","test4")
 
       var targetOk = 0
-      val t1 = actor() receive {
+      val t1: Actor = actor {
         case `testMsg1` => targetOk += 2
         case `testMsg2` => targetOk += 4
       }
 
-      val t2 = actor() receive {
+      val t2: Actor = actor {
           case `testMsg3` => targetOk += 8
       }
 
@@ -48,7 +48,7 @@ class ActorPatternsTest extends junit.framework.TestCase with Suite with MustMat
   @Test def testLogger = verify(new TestActor {
     def test = {
       val msgs = new HashSet[Any]
-      val t1 = actor() receive {
+      val t1: Actor = actor {
         case _ =>
       }
       val l = loggerActor(t1,(x) => msgs += x)
diff --git a/akka-patterns/src/test/scala/AgentTest.scala b/akka-patterns/src/test/scala/AgentTest.scala
index 17ccce8e0a..25961f8222 100644
--- a/akka-patterns/src/test/scala/AgentTest.scala
+++ b/akka-patterns/src/test/scala/AgentTest.scala
@@ -7,18 +7,23 @@ import org.scalatest.junit.JUnitRunner
 import org.scalatest.matchers.MustMatchers
 import org.junit.{Test}
 
+/*
 @RunWith(classOf[JUnitRunner])
 class AgentTest extends junit.framework.TestCase with Suite with MustMatchers with ActorTestUtil with Logging {
-  @Test def testAgent = verify(new TestActor {
-     def test = {
-        val t = Agent(5)
-        handle(t){
-        t.update( _ + 1 )
-        t.update( _ * 2 )
 
-        val r = t()
-        r must be (12)
-       }
-     }
+  @Test def testAgent = verify(new TestActor {
+    def test = {
+      atomic {
+        val t = Agent(5)
+        handle(t) {
+          t.update(_ + 1)
+          t.update(_ * 2)
+
+          val r = t()
+          r must be(12)
+        }
+      }
+    }
   })
-}
\ No newline at end of file
+}
+*/
\ No newline at end of file
diff --git a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala
index dc55e0eca1..b4ea8fc381 100644
--- a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala
+++ b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala
@@ -4,7 +4,7 @@
 
 package se.scalablesolutions.akka.state
 
-import se.scalablesolutions.akka.stm.TransactionManagement.currentTransaction
+import se.scalablesolutions.akka.stm.TransactionManagement.transaction
 import se.scalablesolutions.akka.collection._
 import se.scalablesolutions.akka.util.Logging
 
@@ -64,10 +64,6 @@ trait Storage {
     throw new UnsupportedOperationException
 }
 
-
-
-
-
 /**
  * Implementation of PersistentMap for every concrete 
  * storage will have the same workflow. This abstracts the workflow.
@@ -162,8 +158,8 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
   }
 
   private def register = {
-    if (currentTransaction.get.isEmpty) throw new NoTransactionInScopeException
-    currentTransaction.get.get.register(uuid, this)
+    if (transaction.get.isEmpty) throw new NoTransactionInScopeException
+    transaction.get.get.register(uuid, this)
   }
 }
 
@@ -236,8 +232,8 @@ trait PersistentVector[T] extends RandomAccessSeq[T] with Transactional with Com
   def length: Int = storage.getVectorStorageSizeFor(uuid) + newElems.length
 
   private def register = {
-    if (currentTransaction.get.isEmpty) throw new NoTransactionInScopeException
-    currentTransaction.get.get.register(uuid, this)
+    if (transaction.get.isEmpty) throw new NoTransactionInScopeException
+    transaction.get.get.register(uuid, this)
   }
 }
 
@@ -272,8 +268,8 @@ trait PersistentRef[T] extends Transactional with Committable {
   }
 
   private def register = {
-    if (currentTransaction.get.isEmpty) throw new NoTransactionInScopeException
-    currentTransaction.get.get.register(uuid, this)
+    if (transaction.get.isEmpty) throw new NoTransactionInScopeException
+    transaction.get.get.register(uuid, this)
   }
 }
 
@@ -397,7 +393,7 @@ trait PersistentQueue[A] extends scala.collection.mutable.Queue[A]
     throw new UnsupportedOperationException("dequeueAll not supported")
 
   private def register = {
-    if (currentTransaction.get.isEmpty) throw new NoTransactionInScopeException
-    currentTransaction.get.get.register(uuid, this)
+    if (transaction.get.isEmpty) throw new NoTransactionInScopeException
+    transaction.get.get.register(uuid, this)
   }
 }
diff --git a/akka.iml b/akka.iml
index 2f07a75716..a39c87020f 100644
--- a/akka.iml
+++ b/akka.iml
@@ -2,7 +2,18 @@
 
   
     
-      
+      
+        
+        
+      
     
   
   
diff --git a/config/akka-reference.conf b/config/akka-reference.conf
index 749b599e0b..296b06428b 100644
--- a/config/akka-reference.conf
+++ b/config/akka-reference.conf
@@ -30,8 +30,10 @@
  
   
     service = on
-    max-nr-of-retries = 100
-    distributed = off           # not implemented yet
+    fair = on                     # should transactions be fair or non-fair (non fair yield better performance)
+    max-nr-of-retries = 1000      # max nr of retries of a failing transaction before giving up
+    timeout = 10000               # transaction timeout; if transaction have not committed within the timeout then it is aborted
+    distributed = off             # not implemented yet
   
  
   
diff --git a/pom.xml b/pom.xml
index d2885abf61..480ad8723a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -161,6 +161,11 @@
       Codehaus Maven Repository
       http://repository.codehaus.org
     
+    
+      snapshots.repository.codehaus.org
+      Codehaus Maven Snapshot Repository
+      http://snapshots.repository.codehaus.org
+    
     
       maven2-repository.dev.java.net
       Java.net Repository for Maven
@@ -301,11 +306,11 @@
         
           
             **/*Test.java
-            
+            **/*Spec.java
           
-          
+          
           
             
               akka.home

From f3a457d4de56858be02b040681237b462af64689 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= 
Date: Mon, 1 Mar 2010 22:03:17 +0100
Subject: [PATCH 05/15] improved perf with 25 % + renamed FutureResult ->
 Future + Added lightweight future factory method

---
 .../src/main/scala/actor/ActiveObject.scala   |  4 +-
 akka-core/src/main/scala/actor/Actor.scala    | 19 +++----
 .../ExecutorBasedEventDrivenDispatcher.scala  | 13 ++++-
 .../src/main/scala/dispatch/Future.scala      | 38 +++++++++----
 .../src/main/scala/dispatch/Reactor.scala     |  2 +-
 .../scala/dispatch/ThreadPoolBuilder.scala    |  2 +-
 .../src/main/scala/remote/RemoteClient.scala  | 14 ++---
 .../src/main/scala/stm/DataFlowVariable.scala |  4 +-
 .../src/test/scala/ActorRegistryTest.scala    | 56 +------------------
 9 files changed, 60 insertions(+), 92 deletions(-)

diff --git a/akka-core/src/main/scala/actor/ActiveObject.scala b/akka-core/src/main/scala/actor/ActiveObject.scala
index 1858952f40..d88f0e861b 100644
--- a/akka-core/src/main/scala/actor/ActiveObject.scala
+++ b/akka-core/src/main/scala/actor/ActiveObject.scala
@@ -6,7 +6,7 @@ package se.scalablesolutions.akka.actor
 
 import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.RemoteRequest
 import se.scalablesolutions.akka.remote.{RemoteProtocolBuilder, RemoteClient, RemoteRequestIdFactory}
-import se.scalablesolutions.akka.dispatch.{MessageDispatcher, FutureResult}
+import se.scalablesolutions.akka.dispatch.{MessageDispatcher, Future}
 import se.scalablesolutions.akka.config.ScalaConfig._
 import se.scalablesolutions.akka.serialization.Serializer
 import se.scalablesolutions.akka.util._
@@ -299,7 +299,7 @@ private[akka] sealed class ActiveObjectAspect {
     }
   }
 
-  private def getResultOrThrowException[T](future: FutureResult): Option[T] =
+  private def getResultOrThrowException[T](future: Future): Option[T] =
     if (future.exception.isDefined) {
       val (_, cause) = future.exception.get
       throw cause
diff --git a/akka-core/src/main/scala/actor/Actor.scala b/akka-core/src/main/scala/actor/Actor.scala
index a1eb6bd309..ea230b7d45 100644
--- a/akka-core/src/main/scala/actor/Actor.scala
+++ b/akka-core/src/main/scala/actor/Actor.scala
@@ -240,7 +240,7 @@ trait Actor extends TransactionManagement {
    * But it can be used for advanced use-cases when one might want to store away the future and
    * resolve it later and/or somewhere else.
    */
-  protected var senderFuture: Option[CompletableFutureResult] = None
+  protected var senderFuture: Option[CompletableFuture] = None
 
   // ====================================
   // ==== USER CALLBACKS TO OVERRIDE ====
@@ -533,7 +533,10 @@ trait Actor extends TransactionManagement {
    */
   def !![T](message: Any): Option[T] = !![T](message, timeout)
 
-  def !!!(message: Any): FutureResult = {
+  /**
+   * FIXME document !!!
+   */
+  def !!!(message: Any): Future = {
     if (_isKilled) throw new ActorKilledException("Actor [" + toString + "] has been killed, can't respond to messages")
     if (_isRunning) {
       postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, None)
@@ -541,12 +544,6 @@ trait Actor extends TransactionManagement {
       "Actor has not been started, you need to invoke 'actor.start' before using it")
   }
   
-  /**
-   * This method is evil and has been removed. Use '!!' with a timeout instead.
-   */
-  def !?[T](message: Any): T = throw new UnsupportedOperationException(
-    "'!?' is evil and has been removed. Use '!!' with a timeout instead")
-
   /**
    * Forwards the message and passes the original sender actor as the sender.
    * 

@@ -832,7 +829,7 @@ trait Actor extends TransactionManagement { protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( message: Any, timeout: Long, - senderFuture: Option[CompletableFutureResult]): CompletableFutureResult = { + senderFuture: Option[CompletableFuture]): CompletableFuture = { if (_remoteAddress.isDefined) { val requestBuilder = RemoteRequest.newBuilder .setId(RemoteRequestIdFactory.nextId) @@ -850,7 +847,7 @@ trait Actor extends TransactionManagement { else throw new IllegalStateException("Expected a future from remote call to actor " + toString) } else { val future = if (senderFuture.isDefined) senderFuture.get - else new DefaultCompletableFutureResult(timeout) + else new DefaultCompletableFuture(timeout) val invocation = new MessageInvocation(this, message, Some(future), None, currentTransaction.get) if (_isEventBased) { _mailbox.add(invocation) @@ -937,7 +934,7 @@ trait Actor extends TransactionManagement { } } - private def getResultOrThrowException[T](future: FutureResult): Option[T] = + private def getResultOrThrowException[T](future: Future): Option[T] = if (future.exception.isDefined) throw future.exception.get._2 else future.result.asInstanceOf[Option[T]] diff --git a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index 7da13a10b3..d35b8205d1 100644 --- a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -62,15 +62,22 @@ class ExecutorBasedEventDrivenDispatcher(_name: String) extends MessageDispatche def dispatch(invocation: MessageInvocation) = if (active) { executor.execute(new Runnable() { def run = { - invocation.receiver.synchronized { + var messageInvocation = invocation.receiver._mailbox.poll + while (messageInvocation != null) { + messageInvocation.invoke + messageInvocation = invocation.receiver._mailbox.poll + } + } + /* invocation.receiver.synchronized { val messages = invocation.receiver._mailbox.iterator while (messages.hasNext) { - messages.next.asInstanceOf[MessageInvocation].invoke + messages.next.invoke messages.remove } } } - }) + */ + }) } else throw new IllegalStateException("Can't submit invocations to dispatcher since it's not started") def start = if (!active) { diff --git a/akka-core/src/main/scala/dispatch/Future.scala b/akka-core/src/main/scala/dispatch/Future.scala index c1e61695b8..0dcc0f850c 100644 --- a/akka-core/src/main/scala/dispatch/Future.scala +++ b/akka-core/src/main/scala/dispatch/Future.scala @@ -2,22 +2,37 @@ * Copyright (C) 2009-2010 Scalable Solutions AB */ -/** - * Based on code from the actorom actor framework by Sergio Bossa [http://code.google.com/p/actorom/]. - */ package se.scalablesolutions.akka.dispatch import java.util.concurrent.locks.ReentrantLock -import java.util.concurrent.{SynchronousQueue, TimeUnit} +import java.util.concurrent.TimeUnit class FutureTimeoutException(message: String) extends RuntimeException(message) object Futures { - def awaitAll(futures: List[FutureResult]): Unit = futures.foreach(_.await) - def awaitOne(futures: List[FutureResult]): FutureResult = { - var future: Option[FutureResult] = None + /** + *

+   * val future = Futures.future(1000) {
+   *  ... // do stuff
+   * }
+   * 
+ */ + def future(timeout: Long)(body: => Any): Future = { + val promise = new DefaultCompletableFuture(timeout) + try { + promise completeWithResult body + } catch { + case e => promise completeWithException (None, e) + } + promise + } + + def awaitAll(futures: List[Future]): Unit = futures.foreach(_.await) + + def awaitOne(futures: List[Future]): Future = { + var future: Option[Future] = None do { future = futures.find(_.isCompleted) } while (future.isEmpty) @@ -25,7 +40,7 @@ object Futures { } /* - def awaitEither(f1: FutureResult, f2: FutureResult): Option[Any] = { + def awaitEither(f1: Future, f2: Future): Option[Any] = { import Actor.Sender.Self import Actor.{spawn, actor} @@ -54,7 +69,7 @@ object Futures { */ } -sealed trait FutureResult { +sealed trait Future { def await def awaitBlocking def isCompleted: Boolean @@ -64,12 +79,13 @@ sealed trait FutureResult { def exception: Option[Tuple2[AnyRef, Throwable]] } -trait CompletableFutureResult extends FutureResult { +trait CompletableFuture extends Future { def completeWithResult(result: Any) def completeWithException(toBlame: AnyRef, exception: Throwable) } -class DefaultCompletableFutureResult(timeout: Long) extends CompletableFutureResult { +// Based on code from the actorom actor framework by Sergio Bossa [http://code.google.com/p/actorom/]. +class DefaultCompletableFuture(timeout: Long) extends CompletableFuture { private val TIME_UNIT = TimeUnit.MILLISECONDS def this() = this(0) diff --git a/akka-core/src/main/scala/dispatch/Reactor.scala b/akka-core/src/main/scala/dispatch/Reactor.scala index f7bfa52215..bf8254c64a 100644 --- a/akka-core/src/main/scala/dispatch/Reactor.scala +++ b/akka-core/src/main/scala/dispatch/Reactor.scala @@ -14,7 +14,7 @@ import java.util.concurrent.ConcurrentHashMap final class MessageInvocation(val receiver: Actor, val message: Any, - val future: Option[CompletableFutureResult], + val future: Option[CompletableFuture], val sender: Option[Actor], val tx: Option[Transaction]) { if (receiver eq null) throw new IllegalArgumentException("receiver is null") diff --git a/akka-core/src/main/scala/dispatch/ThreadPoolBuilder.scala b/akka-core/src/main/scala/dispatch/ThreadPoolBuilder.scala index cb465907cb..1fedc1a5d7 100644 --- a/akka-core/src/main/scala/dispatch/ThreadPoolBuilder.scala +++ b/akka-core/src/main/scala/dispatch/ThreadPoolBuilder.scala @@ -4,11 +4,11 @@ package se.scalablesolutions.akka.dispatch +import java.util.Collection import java.util.concurrent._ import atomic.{AtomicLong, AtomicInteger} import ThreadPoolExecutor.CallerRunsPolicy -import java.util.Collection import se.scalablesolutions.akka.util.Logging trait ThreadPoolBuilder { diff --git a/akka-core/src/main/scala/remote/RemoteClient.scala b/akka-core/src/main/scala/remote/RemoteClient.scala index f97f014f06..0887ebcd82 100644 --- a/akka-core/src/main/scala/remote/RemoteClient.scala +++ b/akka-core/src/main/scala/remote/RemoteClient.scala @@ -6,7 +6,7 @@ package se.scalablesolutions.akka.remote import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.{RemoteRequest, RemoteReply} import se.scalablesolutions.akka.actor.{Exit, Actor} -import se.scalablesolutions.akka.dispatch.{DefaultCompletableFutureResult, CompletableFutureResult} +import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture} import se.scalablesolutions.akka.util.{UUID, Logging} import se.scalablesolutions.akka.Config.config @@ -86,7 +86,7 @@ object RemoteClient extends Logging { override def postMessageToMailboxAndCreateFutureResultWithTimeout( message: Any, timeout: Long, - senderFuture: Option[CompletableFutureResult]): CompletableFutureResult = { + senderFuture: Option[CompletableFuture]): CompletableFuture = { val requestBuilder = RemoteRequest.newBuilder .setId(RemoteRequestIdFactory.nextId) .setTarget(className) @@ -168,7 +168,7 @@ class RemoteClient(hostname: String, port: Int) extends Logging { val name = "RemoteClient@" + hostname + "::" + port @volatile private[remote] var isRunning = false - private val futures = new ConcurrentHashMap[Long, CompletableFutureResult] + private val futures = new ConcurrentHashMap[Long, CompletableFuture] private val supervisors = new ConcurrentHashMap[String, Actor] private val channelFactory = new NioClientSocketChannelFactory( @@ -208,14 +208,14 @@ class RemoteClient(hostname: String, port: Int) extends Logging { } } - def send(request: RemoteRequest, senderFuture: Option[CompletableFutureResult]): Option[CompletableFutureResult] = if (isRunning) { + def send(request: RemoteRequest, senderFuture: Option[CompletableFuture]): Option[CompletableFuture] = if (isRunning) { if (request.getIsOneWay) { connection.getChannel.write(request) None } else { futures.synchronized { val futureResult = if (senderFuture.isDefined) senderFuture.get - else new DefaultCompletableFutureResult(request.getTimeout) + else new DefaultCompletableFuture(request.getTimeout) futures.put(request.getId, futureResult) connection.getChannel.write(request) Some(futureResult) @@ -238,7 +238,7 @@ class RemoteClient(hostname: String, port: Int) extends Logging { * @author Jonas Bonér */ class RemoteClientPipelineFactory(name: String, - futures: ConcurrentMap[Long, CompletableFutureResult], + futures: ConcurrentMap[Long, CompletableFuture], supervisors: ConcurrentMap[String, Actor], bootstrap: ClientBootstrap, remoteAddress: SocketAddress, @@ -269,7 +269,7 @@ class RemoteClientPipelineFactory(name: String, */ @ChannelPipelineCoverage {val value = "all"} class RemoteClientHandler(val name: String, - val futures: ConcurrentMap[Long, CompletableFutureResult], + val futures: ConcurrentMap[Long, CompletableFuture], val supervisors: ConcurrentMap[String, Actor], val bootstrap: ClientBootstrap, val remoteAddress: SocketAddress, diff --git a/akka-core/src/main/scala/stm/DataFlowVariable.scala b/akka-core/src/main/scala/stm/DataFlowVariable.scala index aa5a8255e4..daed4ec55f 100644 --- a/akka-core/src/main/scala/stm/DataFlowVariable.scala +++ b/akka-core/src/main/scala/stm/DataFlowVariable.scala @@ -8,7 +8,7 @@ import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue} import se.scalablesolutions.akka.actor.Actor -import se.scalablesolutions.akka.dispatch.CompletableFutureResult +import se.scalablesolutions.akka.dispatch.CompletableFuture /** * Implements Oz-style dataflow (single assignment) variables. @@ -74,7 +74,7 @@ object DataFlow { private class Out[T <: Any](dataFlow: DataFlowVariable[T]) extends Actor { timeout = TIME_OUT start - private var readerFuture: Option[CompletableFutureResult] = None + private var readerFuture: Option[CompletableFuture] = None def receive = { case Get => val ref = dataFlow.value.get diff --git a/akka-core/src/test/scala/ActorRegistryTest.scala b/akka-core/src/test/scala/ActorRegistryTest.scala index a751acee3e..ada0c027d5 100644 --- a/akka-core/src/test/scala/ActorRegistryTest.scala +++ b/akka-core/src/test/scala/ActorRegistryTest.scala @@ -3,58 +3,6 @@ package se.scalablesolutions.akka.actor import org.scalatest.junit.JUnitSuite import org.junit.Test -/* -class ActorRegistryTest extends JUnitSuite { - - val registry = ActorRegistry - - @Test - def testRegistrationWithDefaultId { - val actor = new TestActor1 - assertEquals(actor.getClass.getName, actor.getId) - testRegistration(actor, classOf[TestActor1]) - } - - @Test - def testRegistrationWithCustomId { - val actor = new TestActor2 - assertEquals("customid", actor.getId) - testRegistration(actor, classOf[TestActor2]) - } - - private def testRegistration[T <: Actor](actor: T, actorClass: Class[T]) { - assertEquals("non-started actor registered", Nil, registry.actorsFor(actorClass)) - assertEquals("non-started actor registered", Nil, registry.actorsFor(actor.getId)) - assertEquals("non-started actor registered", None, registry.actorFor(actor.uuid)) - actor.start - assertEquals("actor not registered", List(actor), registry.actorsFor(actorClass)) - assertEquals("actor not registered", List(actor), registry.actorsFor(actor.getId)) - assertEquals("actor not registered", Some(actor), registry.actorFor(actor.uuid)) - actor.stop - assertEquals("stopped actor registered", Nil, registry.actorsFor(actorClass)) - assertEquals("stopped actor registered", Nil, registry.actorsFor(actor.getId)) - assertEquals("stopped actor registered", None, registry.actorFor(actor.uuid)) - } - -} - -class TestActor1 extends Actor { - - // use default id - - protected def receive = null - -} - -class TestActor2 extends Actor { - - id = "customid" - - protected def receive = null - -} - - */ class ActorRegistryTest extends JUnitSuite { var record = "" class TestActor extends Actor { @@ -62,6 +10,7 @@ class ActorRegistryTest extends JUnitSuite { def receive = { case "ping" => record = "pong" + record + reply("got ping") } } @@ -180,8 +129,7 @@ class ActorRegistryTest extends JUnitSuite { val actor2 = new TestActor actor2.start record = "" - ActorRegistry.foreach(actor => actor send "ping") - Thread.sleep(1000) + ActorRegistry.foreach(actor => actor !! "ping") assert(record === "pongpong") actor1.stop actor2.stop From 9cfefa09d67e1cde33bb652310e455715abccd05 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 2 Mar 2010 00:20:23 +0100 Subject: [PATCH 06/15] Fix for link(..) --- akka-core/src/main/scala/actor/Actor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-core/src/main/scala/actor/Actor.scala b/akka-core/src/main/scala/actor/Actor.scala index a1eb6bd309..39a354a831 100644 --- a/akka-core/src/main/scala/actor/Actor.scala +++ b/akka-core/src/main/scala/actor/Actor.scala @@ -656,9 +656,9 @@ trait Actor extends TransactionManagement { * To be invoked from within the actor itself. */ protected[this] def link(actor: Actor) = { - getLinkedActors.add(actor) if (actor._supervisor.isDefined) throw new IllegalStateException( "Actor can only have one supervisor [" + actor + "], e.g. link(actor) fails") + getLinkedActors.add(actor) actor._supervisor = Some(this) Actor.log.debug("Linking actor [%s] to actor [%s]", actor, this) } From 10aaf553a1217f26531437d40de47f2bc8a0400c Mon Sep 17 00:00:00 2001 From: Debasish Ghosh Date: Tue, 2 Mar 2010 12:14:29 +0530 Subject: [PATCH 07/15] upgraded redisclient to 1.1 - api changes, refactorings --- .../akka-persistence-redis/pom.xml | 2 +- .../src/main/scala/RedisStorageBackend.scala | 57 ++++++++++-------- .../redisclient/1.0.1/redisclient-1.0.1.jar | Bin 50730 -> 0 bytes .../redis/redisclient/1.1/redisclient-1.1.jar | Bin 0 -> 47464 bytes .../redisclient-1.1.pom} | 4 +- 5 files changed, 36 insertions(+), 27 deletions(-) delete mode 100644 embedded-repo/com/redis/redisclient/1.0.1/redisclient-1.0.1.jar create mode 100644 embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar rename embedded-repo/com/redis/redisclient/{1.0.1/redisclient-1.0.1.pom => 1.1/redisclient-1.1.pom} (89%) diff --git a/akka-persistence/akka-persistence-redis/pom.xml b/akka-persistence/akka-persistence-redis/pom.xml index c6088e573b..112d4764cb 100644 --- a/akka-persistence/akka-persistence-redis/pom.xml +++ b/akka-persistence/akka-persistence-redis/pom.xml @@ -24,7 +24,7 @@ com.redis redisclient - 1.0.1 + 1.1 diff --git a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala index 00a44d0513..48945d6b8c 100644 --- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala +++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala @@ -15,10 +15,10 @@ trait Encoder { } trait CommonsCodecBase64 { - val base64 = new org.apache.commons.codec.binary.Base64 - - def encode(bytes: Array[Byte]): Array[Byte] = base64.encode(bytes) - def decode(bytes: Array[Byte]): Array[Byte] = base64.decode(bytes) + import org.apache.commons.codec.binary.Base64._ + + def encode(bytes: Array[Byte]): Array[Byte] = encodeBase64(bytes) + def decode(bytes: Array[Byte]): Array[Byte] = decodeBase64(bytes) } object Base64Encoder extends Encoder with CommonsCodecBase64 @@ -45,7 +45,7 @@ private [akka] object RedisStorageBackend extends val REDIS_SERVER_HOSTNAME = config.getString("akka.storage.redis.hostname", "127.0.0.1") val REDIS_SERVER_PORT = config.getInt("akka.storage.redis.port", 6379) - val db = new Redis(REDIS_SERVER_HOSTNAME, REDIS_SERVER_PORT) + val db = new RedisClient(REDIS_SERVER_HOSTNAME, REDIS_SERVER_PORT) /** * Map storage in Redis. @@ -189,7 +189,7 @@ private [akka] object RedisStorageBackend extends } def insertVectorStorageEntryFor(name: String, element: Array[Byte]) { - db.pushHead(new String(encode(name.getBytes)), new String(element)) + db.lpush(new String(encode(name.getBytes)), new String(element)) } def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) { @@ -197,11 +197,11 @@ private [akka] object RedisStorageBackend extends } def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) { - db.listSet(new String(encode(name.getBytes)), index, new String(elem)) + db.lset(new String(encode(name.getBytes)), index, new String(elem)) } def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = { - db.listIndex(new String(encode(name.getBytes)), index) match { + db.lindex(new String(encode(name.getBytes)), index) match { case None => throw new Predef.NoSuchElementException(name + " does not have element at " + index) case Some(e) => e.getBytes @@ -221,16 +221,16 @@ private [akka] object RedisStorageBackend extends if (f >= s) Math.min(count, (f - s)) else count } else count - db.listRange(new String(encode(name.getBytes)), s, s + cnt - 1) match { + db.lrange(new String(encode(name.getBytes)), s, s + cnt - 1) match { case None => throw new Predef.NoSuchElementException(name + " does not have elements in the range specified") case Some(l) => - l map (_.getBytes) + l map (_.get.getBytes) } } def getVectorStorageSizeFor(name: String): Int = { - db.listLength(new String(encode(name.getBytes))) match { + db.llen(new String(encode(name.getBytes))) match { case None => throw new Predef.NoSuchElementException(name + " not present") case Some(l) => l @@ -251,12 +251,12 @@ private [akka] object RedisStorageBackend extends // add to the end of the queue def enqueue(name: String, item: Array[Byte]): Boolean = { - db.pushTail(new String(encode(name.getBytes)), new String(item)) + db.rpush(new String(encode(name.getBytes)), new String(item)) } // pop from the front of the queue def dequeue(name: String): Option[Array[Byte]] = { - db.popHead(new String(encode(name.getBytes))) match { + db.lpop(new String(encode(name.getBytes))) match { case None => throw new Predef.NoSuchElementException(name + " not present") case Some(s) => @@ -266,7 +266,7 @@ private [akka] object RedisStorageBackend extends // get the size of the queue def size(name: String): Int = { - db.listLength(new String(encode(name.getBytes))) match { + db.llen(new String(encode(name.getBytes))) match { case None => throw new Predef.NoSuchElementException(name + " not present") case Some(l) => l @@ -277,40 +277,49 @@ private [akka] object RedisStorageBackend extends // start is the item to begin, count is how many items to return def peek(name: String, start: Int, count: Int): List[Array[Byte]] = count match { case 1 => - db.listIndex(new String(encode(name.getBytes)), start) match { + db.lindex(new String(encode(name.getBytes)), start) match { case None => throw new Predef.NoSuchElementException("No element at " + start) case Some(s) => List(s.getBytes) } case n => - db.listRange(new String(encode(name.getBytes)), start, start + count - 1) match { + db.lrange(new String(encode(name.getBytes)), start, start + count - 1) match { case None => throw new Predef.NoSuchElementException( "No element found between " + start + " and " + (start + count - 1)) case Some(es) => - es.map(_.getBytes) + es.map(_.get.getBytes) } } // completely delete the queue def remove(name: String): Boolean = { - db.delete(new String(encode(name.getBytes))) + db.delete(new String(encode(name.getBytes))) match { + case Some(1) => true + case _ => false + } } // add item to sorted set identified by name def zadd(name: String, zscore: String, item: Array[Byte]): Boolean = { - db.zAdd(new String(encode(name.getBytes)), zscore, new String(item)) + db.zadd(new String(encode(name.getBytes)), zscore, new String(item)) match { + case Some(1) => true + case _ => false + } } // remove item from sorted set identified by name def zrem(name: String, item: Array[Byte]): Boolean = { - db.zRem(new String(encode(name.getBytes)), new String(item)) + db.zrem(new String(encode(name.getBytes)), new String(item)) match { + case Some(1) => true + case _ => false + } } // cardinality of the set identified by name def zcard(name: String): Int = { - db.zCard(new String(encode(name.getBytes))) match { + db.zcard(new String(encode(name.getBytes))) match { case None => throw new Predef.NoSuchElementException(name + " not present") case Some(l) => l @@ -318,7 +327,7 @@ private [akka] object RedisStorageBackend extends } def zscore(name: String, item: Array[Byte]): String = { - db.zScore(new String(encode(name.getBytes)), new String(item)) match { + db.zscore(new String(encode(name.getBytes)), new String(item)) match { case None => throw new Predef.NoSuchElementException(new String(item) + " not present") case Some(s) => s @@ -326,11 +335,11 @@ private [akka] object RedisStorageBackend extends } def zrange(name: String, start: Int, end: Int): List[Array[Byte]] = { - db.zRange(new String(encode(name.getBytes)), start.toString, end.toString, SocketOperations.ASC, false) match { + db.zrange(new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC, false) match { case None => throw new Predef.NoSuchElementException(name + " not present") case Some(s) => - s.map(_.getBytes) + s.map(_.get.getBytes) } } diff --git a/embedded-repo/com/redis/redisclient/1.0.1/redisclient-1.0.1.jar b/embedded-repo/com/redis/redisclient/1.0.1/redisclient-1.0.1.jar deleted file mode 100644 index ec9c6c8ab44d2552946c18f1ae0daee03f2570c4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 50730 zcmWIWW@Zs#;9%fjSUyF=h5-qPFt9NAx`sIFdiuHP`#So0y1532==r++JH^28+4sz8 zA8%c~i@e^tTIbH3-yCFc#rVO~M^Bj;0=(HdHq|q|lV)IGkYr$B2!LC#6wLyC1_p-Y z{9OH_)RfF({rrN|qQsKS{JdiQ;QXQze>hhqIVZ8WSPx824b2V~3Y4hhTi5Wqt7W>& z?uy7nJf|IPLI9L>3zS+Cqp_oj;nPE>2Y zSzLSX`=0Z~^IGHg+t)K1G_bN6E*1T{#b{-kXxp)l;s&ty|_ym3(7o zx?IYRZNL4zYi7#3CIxZ}uI=fb!{5#NOtPg=*I82fah9K_h3^eex5Ot&7m zP`omPXWkCC&@-oZoqFu;`9`vrSyD_tJg_FVPTze~QJ(1}qY~5YQK_l|<}ckQRmwT5 zm&Yq@W)^Zjyeg?Dd*_@<&$4+O>$kpHe$#KGA*Y_nwZ)8iY!(j_3+A_}gd7YFO>u2H6PIr5yuP+YEvbo8yy)1~GXk56};akCP z?Yz)K0`Cfc)J{rt{(0hWsiJmsbJndXJNc4+ojJNKY`;rU^}&+ISL%0fbh^3eyGpiQ z_?h&nj_mU$Y&p$5HECLts?E_E9f3W?R~EEzW?NOB^AR&=pJ&Vy8u)4c`(_qF{T;#a zK~dT1Zf2LVg?Fu+xIgV*lJDgVPeIN-N;6Yxw-&^G66%|M;nzma7$?VgD{b);viU7~ zXC}Y?xty)^3&#Q<6Yh(Le|+wgl74gL)>&5dgA6BcwKP9lAS(EE>4EuETE0ggyui5b z#srS)S&I%m;`iW=duSbF%~9QITe1D&YipO|kL>cQc73}byy$rG>{+$}4;px_IpSFE zELDr?d}Dopx$SPz!nJ9PXG5a6_eSMPET3AsWp*ojr=n=M)|`I1M-x5MxC5nbW=wJC zIlN4+-!N#=*WPD*`^0B)$|sjd8$VR;QTz4ngN{~!wdmTfT*qhJb336elW3Lo{o1U{ z%N9&;+fvj}!8`GjywXOAd&gg|f9iOHWmof*!dsV99+jP*of@_`?XTXi>zPZ+FSu?s zo0K0L(IlO}GJe@V*{`L`n8f;pr&XR&a`nkk3zn{Z{^OH8D4)iE4HffdW?(qMhA*EQ zqU6)y{N(IZ;n;|o_Gy}K-Q^1jDr-CMG&8b$05zwCOuRX5&b&bq7oOW%meT%Q;& z!8LO+&ttt!**wbKam^ZX>n+wraNe4IX#Hnj*Q&hxHL4j~*H!j;r%cX>Ak z@l8`^sd*O@A4oX=EBLaP#+E z(OEMt8m|7Z-1+XBrw6u*W*poSz32ATf}qZ&E(VgzjHcSHUZnI>$M3a8tFg$&J4L&A z#QQF2xoumq?u7CU(|6Wh^F5ZZyf2CRE_p6h>+hQGn?Hm$a$lD@zlv>>u*WWrgSVwR zzMte+l&`PypXuC7m2=)aoA=)Mb6~1pbcAYtv`Ls|0dHhiPU+GZ^(Chi-CUU~=g2$r ziNAQa?YD4#f66NRV3P?iGlEZCs@3$^J<+FZams_xJ+IV%Itmy9* zU#i4col>!0@%#+`BALy`1;X1!pRvVOuDyP`$LQ{&{VQBTH7C8&SA1nydqQfT!Yjk4 zra5veCeK>Xlx-RR{oTHaS^=_Wg0?@{plC2%bhzNzrFVc-8br|l1=y!)f<^IYz+%RA0za+6L9lt@i35%?i{{<4L0cFB}^n;m{E zjoEOebk%WjWp?k0%_Z-ued5kOtv~pO&*=WcN6pF#mjzRrb}eh2o@u|<%cd#kT@jz+ zXA{5o55CIA{9`+Q?dmQ5is^}e6=$j&c=~G=nJjEMFn`M@_GKT$pJntNpSU19`pEB+ zS%SY)Zw33EZ~4E)yQt=mZtaHp!uW-SE&JAAiaxrv<7KMVngx@0{t>>?*JZI&J!z8C zy7qHn?2p&{kDoJphv-8Yo=20}H!8Fpj@MR`q}vSrFLR&6c6$?nz$GdVXqba(A||9jtqw%scN6t_IOc>jyce~td{zL!i+ z9+9)Pt({!?KJR|{``Yq1blG=6iLh$W?>hs{)nlbW=0tlvgIavGBJN znsz!uQAhvtkIr|cy-PmaTc}ccb(`uIjxd(2)(FnyrdY)ivi@(u{M~ zIQX?1bY2L!qte_S;w?PqN$1K?W69dCL}!aQx%}|i_1%e~dRnJVWpvLj-hI|HkALSe zgS{RLcBEdHFjj9rJL93i1+TDV$D9LhlCK5TL>|s)*vcn3Gb1}a;^FrBQi@(GIchH_ zHr_CB_Wl#!va6}%(KM?ZM$y=>OEmTds0y65{M(W&7`;2z*T2yFkeTFd*DA5+e(aqg zbv_dd*qg7^C!F`#6E3(lw8%@&`eJCi^~PqA(B!PuZzWY9Nt%VLY&H27!*h3~NaU^7 z-6@+&%$Z-uwA{5b;@TwiE#TFZo_DuWSDx;ivP5sc=#jWP%8SK+uQ;Xg`pyE5+hQFX zT4aOXAFgu@n4@ShJ<(22s>1T=h7&$@^G(`dpObrN?3gbrDynm^Ic z;Jk9jS0wb()zJ3uJ|CWN-u9g?dPC;hK81HjGi>=D+iA_5bN$)}nKesRzP|H%MaHdC z7jc*0S9x3u?BxWk_>Hbq{&=)3Acya}=y7c`&u@w=m~KkMRw+HdJ1uqIyKT9~I}0v- zb~V=bTNd;{>E&^*o8LG+XDHZvMH$CLOK#usDt7j3u@A;?(n|JC?U6suDJfObX3D*A zdO|?oo~oc9o{OwvvOKnAu&KXJY_d7MEz{Cw>wN#fucjg$z1iC`9N5<__$cE1_m5ra zMU}7a6I0d|pSzOC#V@`{J2S)4bm9U(&x$FlY-^V2m)qN`J=QsQQP)5`rD^^fE$$@Q zc@s{4nyhr=(7u-CD|d-3I$+WjzU#`Nqpwo8%{y9bH1FW&mQuMNf&I%>msW|~>#p!z zP(O1WPj`BWu4nRuqc4oC4!)e0S{pKEwD$PDuRqo9 zP5sDsFFE4n)P||oGB}p`wu}32;peD+I8{h>rCU}-lW^41Lm4068svQL{>#2s;oM_k z!$o$Y$@lX^7ri?-!;Jrtlixe;SzD$ut*UUZQ8n4OEX(7`K7%I1!wOVBP8ygoe%Tv`oIOO`d4z@`jij&!S*Nd9`(NB~L z54b8JB+tZP`|@&w-82<}BArEB`|b&?SXBJr*f;yYdf9^)vm-uLFZruAOL_LXq{t^y zVQy&!hf{@CHu+>Xh%`>!(RuCZ7WY+Ou5(rFZ{faHF`?_=fO8v2yZ^)0LC!FKA7eBdlhfeY8sUw&thX%QsHGH!by| z?5mSX(_H_*l9XBEE~dQMBB(V_(WZNG&xxBit(sHB9$WT)(RriQ@p#UO%(ip0GHr@v z)7`V{*2=HVlikyHd~?OBrE1;cX&=2rpLu2~bTDoSShISCQCPJ{ln!`Uz zDRNv${a|%Z!Po8XXE9sWWm~_fMXXTWc7MNzeFUee{KY8e{q{@#fBs?N@1m~By^E>o z+IcZ0lYkR}4NZUU6qhaBG*|!sdReLImDWcU_U(Ma9Qf9@b#=b2@coPR9rsLPj__wJ zmXSZ!-zy8L5gyr1Y1_%lz;KltUyY!LQX`;s=TcLGksFPWHsapUXn)~Afq66c&TF04 zs;5=qxFBm|)?R_YgVGW#E~|C!tz?`Z`n_oV@U-l6y^Di3nB1GT^w{*%>pi}GoOo8? z;cLGOerpyUy*hPMebu75KfM1)XL=gNuR5Zuk;2ea;O*TgzTV6A%PuF5)3@I)6wrIF zv_E3ilRFzeEUOg1RFwWH@M8Zl`P_?dKL0;+eVR^HX|9*sZLM5ycjMa}*^xX=VRJt% zQaTjCeM-0T_`jD|=A@c+{8bI}SZ26tv!mjyns$S|XYbw!I`ddo@mKW0tkQ$=ZWfp3 zUFuIca8>DbMcwXD!`4ua<^%dCHVJ%7aht_fuxjI>tuxol|MvV zO01Q$55%?n+g~lUA%5PS((;xwZTUAooZ7mBhu5MbA^*#?`k%A9%@Q~k?$)bX`>C`z z%3Y!*(ZS-#_q8jPG!__Lk`&D7;WdzsFxa+Waig`Qy28x^u|2sj@*c&7_5mT(1{9crmU% zzKqYhajux^q!nJgI&YjpwUk!up8Tciwb{}#)e9Rph;N;x*HmVG<)TX8%C86AU%1=G za(NZrI=EM6v;T#<_A9${40{=WeEcW;`mTw|UW<&&7Ydj{9W$q{>0M~J^{`JlOSbc? z6*F_pPS+?m^;xC(v?bpB&NRF4>7*%EdM{d1ORSSGaJN-IIrYlVLPv3#cRR|?Sln(|(9Ne=bHeAX(eaprzwSQ1V3{^E z<+RAU*q*+kwdW3UU*1z7dtHv-eSYG%pFJ_NpYGqeW`VR$=bqf$U6uBp+ux~a?w_&b zb{^CFh%E_=>;zjAYdjYyR@`Q}8+`F!@e@|t3J+=J{Ed@W+CDquU-0?xyNpc@%Z>gU zNBi2IFJjc+ar##<*I%1(G5_m!77Wb0e(-{NZ+ZQT?prf6FnDp`%a+78)Qli`vNz;> z@MQ;)f77xu`!NXcsM)PoE^sHk-%Y0{;6t1docYhye^T1V- z{~zPOz`p#+hY}_?U%q*F&-VLu@Av+__wD=q`TH531f&R@&U-NP+Pc{762HFpJUiL= z_R6$b%L2_Mc3sas{>@zS*wsld%}UO%y5;*uL z#Ux4uddse`QGa>7p+kCtCj0**c}8c{pRCYmdu;Wd$=z_()mpF6$aBS6<{xzXS4=*Y zB%!raQ*s0mJ^S@Ta0h84AV%@hkwiKS}nsG3eO_OKq z(_N*f3nY$RdHmXP``=1Vu4ffHeyhZJpZs{*b4Aa#X`UQM&9fGsV0|@n#SNqR%6FFT z;IG&clw-YKR7J-@d}B*Hk8q;I`o!1lVTZr2ojWaaZsAYX0F95QRGu2AJ*ZSUyJ1!7 z7O%ddV4+{?Y?C)ME#EUgOtPwG`j^G<>og}$*LXfjBiVv8JL35q=_yH;RY5@q z3w82m$G5o7t`y{{AeJ3`|4@2(!#s{PRx!tFmI0X z*XkFn?;eUDQe!*DFY$c&z-+?gM`+l^*-F>`j`o(24tdYRb#i+|7H+Z{Oik zJrkZ;_3d&x$4h;W<@+X`{r52Sif?y`(dBJB%$;4IKDuz+~E`KDlz8sZOs<{JFore-UW)2^4X$qOe?Nu1m%>&xm#o}GBGerX2q9NjB!<9-l>(r z1&PV2*vBN7hGmBfhl|`5XKGXt;pkHGT6j#zLqYVn!@_Mgi#r`Y{~H*4w_J_QDZw;%vT1SpJKOU4wcjkitJMGd`dgks z@37E^r)OhVYvjL3)hsYmod0ZfRAAbwOP6x)nq+O>x+v@#%P!|#3vVknW>`d;Xd8M@ z`x12~=f=^f?Rq|IxhulX2u=Jwd6(kl-BX*kD$xt zY!)qJ((|vKH_yKJ~`tZuF%KT9+!6CgjF1344D0!c7s0 zqQAS0Pw%eUsiLcYZ01bYP~+>YR)@R3vF~_sm?gq_@6L)UO;P{*vhNSJZ@IiGG;>M> zcRTYrk!>qeltX<#^yu|ywwmcGbO?#Q2zYhQDB^ZWE~{VQ6tPo`H7%#JUe9TWf4Mzp zR_aPgzO!c;?2f5>ZSy~QT4#FMli<*DU5yyiC&}qsZ--95S#VfsuIsUkOt+J3RHn4r zZ=Vu(c8SnO-lHG68yh~&p1f77bQ+`1&Y4kZnOB=lwJ&vt9{ctrL!>m4X-DI1yCWY( zi|eP%6_4lsvNIrZS(F`5zES0Rtf;DR^}>9%?;vea2t{GM>Y>~rwdoaiW>qwDtEE=#|qBCa)UtJLoHiy^%0+~PuPg$+J3 zUs*1__eBf)mxY%Od#`wHe$Q&h3Z5%2{AU{)c5~?8KPV~x=WDNd=Gtp+%f1_)5H{M< zKap>T{_h`9uKEF?Z5v5)W5)Z8cDj}qtJ z=iU^&_}ZGw*R1FEO>SQk9N7M~WXhV$SBl=wN^m#kIX)V4`^Q}jmJKVetC+y|9GGxq50Rc2veVCTeF2H~6& z@yRSk9&`oIhCpiUwNu`P35Scc-4}LZQq*W#6tOlSP}rj<;`XOdUHOSF?GBSYMBg5t z;FG9uX36BvrPD8Pzu^5+9v=R?eOlStI0yYztKZL*Jm!$pd*RHila|%rZoV)6cCYx~ zzvun?8GIJ#324qJUA@HiiuSAnm9Fb=YKNW;%DronwLM#P>msfew^HZr?yO0MOH)2(*psO?h|h%q?bQxSHNS2z6r-3x0L?NT(ga(}D4SFK?2+5o1%U7lU0 zS1&!S()ZZmw0+Mu?G1P2CT&`~FC>J1;!9B*^8^i5}%{n&NjtJ$JfLD3g?ue@Qs zY_Mh07S5idQ7kzL)laR3k4nyY>!|;Wg>8;ZHTSdERZG`ICVnw~x!LIDcUG&Uk6Sg) zJ$7PYmPlcmT2ocIgM0dm#bK*w1&Vc;cdgIK3GR5L>lFOk=-$MM>o_>vD#IUadYyii zH~Rk7pwG|dg`K|MtE8UD!|h^zX78ezra8I6n_1YBZ&ozVN>6|Ay}-6faD)G^zI3IY z*-O@D99y>c=^G7cSDud?d5Jy>7o=Z0IW0JC7F%AHq;qf1IgLDJ)tm1HeubuSr)9cZ z2hH1k=F;|Qd7FdvwrFvGuH={h@}Rax<3e&~R^>et?O9nCIj;kKlDziKeOBZC+s`a9 zSiozF&$6D(Gev)b)?U#R*EHR7B;oa^;>DY4yQiJ|H`liCe_HD9wkb(=+Pf8f+h(ne zdh)U$YSHbVE;|}RZU%Q$#BK@tdgHKHo!<5-%bq0Xxjgf8OFWx=JZ5o(n_pUCYvx3Q zuJC_LCtotzSfb@+Rqd9O>({NhwEc~gamn-z>n6^34Br}e zyC`<*WTS0%&z*i=Rd3}9Ewd2~w_Z|_wk<#XpzXiMxi9*rX=U}s-!pf&tNX6PwOCB1 zkyZNB_2#@11&(9SbLMQH@O^y^i@+n}Clb3}u({lw&QZ)4r4;i{@6SxfC-sFne;A%K z&EVr>{eR@Z=KBR3>oerTk5_(=fBg8=`yGWAe(#d+%vQU*()z~nBaC)j2^tj)*FLDv zf65cuP^x9$-XzX<**5%o@}$L)HE#;U3g!8C1ohc1^j)W$!1wu1p1IhD7aRY-*q9!* z!M;ZJ_f@tJ<{zG|3{!C2*l7Ovb6U+ko@qAsc-FI@yQMncyxTqZgI50O-C7?`8{Tsd zsrHF`8nS18(S5!A-ux%KmmJ?c$6|@D+$FOhRlX0A##dyXJ};b{IyvBQo`byBOJ`M? z28O-!wq2d``hRh;Ur=S&%Gq_v#diPupVc&;xBd2h?t0-FpKcVNbpE6tJ<(i9jeo<% zIVY@LW*&%*6Dl~v`;OV~XG5^vgC)&Z9sdVOn_i6zzAUD(T|+9__i^s^FR#rQw%uai z;kGi7sb>rCg5=xGKMrd9>{@9s_1GbmzHNbvTqj1ATy#~6dGyirO2CpW{L{W2yyW-( zgp%$r*IT*F7j0~lK4oyb26DT0B`|V7QJa23b!+!zrBkas zM6>s`vpVVA{OH`VQFu~E$cB$s&&;V#dv+$~|BtUvwHb5{+BAjsetSGChJTh&VwJO8 z$oD_9?%bNZ@jl<84W%nXKhL&nio3s#OpPbiPE%%`?U{$=jeXNuWgAvzS6x96L@AkIrxS9)~$Pq zR?N!;k6DK88?!8)gm+A2Q4QvlZ+Pm9cHIt)BAM za6|AN3!f$Fd~;0NUQSowZ?F%oS;W0$@%eW7gG)Len&||EY+1w@eL2wk_x1}8tycpS zmN%FtUhbHCw8!;kV94ZYx-!nDf(Mp-tWi#D5Cl_S}IjXLQ zGOt=LR!w_W$vwr^`Hcp8OAdD5h!tvo7PGK2=0K{?_R9;GPFnk9U5xR|Rfm0-%s$rl zT6J=s#ges0Hr6LxKKn`IY2M6|+m3%VS1ZTQ-cr;4<(yR18}VDVX?y$srcXLPwPmNu z7SleP1Mj?xrnD~=3g4M`Nx8k}sz#QVy-1WQ>(_fLE|}fe*m-xpgPgp2%Da3}qPBEm z{8Pomz~I4xFHwWmIV8hZF*xVv<)tQrmNBS6l5lTGHmE;s8{O!@6UE~Au{CPRLMO)f z3_(}9DBT5HLO#wt#?z8H*>9G`kNJNX9`mH89Q(N0{?x~JYI|nAnBh~lx%&Oi^~L4& zfB*esKA=&;W|?2zmv?4nYO|tUZuGQ;yO-tiUN2d@#?a*X-^f|!2=5wNh6WA$(EOm z#P)cdEZ~arY}vZ5_S&V?vr)eK)2?yqJzUabA;RjLk*9n^vq*`_?|kJn1N&(gPDia( zTi88EWrr}=UhSFL7f!8kWxe}iVxd&JM)cY!-TFic|7TLQX@{({DqeeRsJw2tHk9+Z zWL53{zVw}GM?LH}nfKh9lrEZ~_)zKMgGklt#R=)U(}OxctVN&^G2QI;CP&3h($BTkf03cyCe3?`p3Ruk*wD64V>(^3{ThqML`=nrb&0w7Spf%WP}> zpDi9ZAyV?Y!FTaVt;d8VAD`gTaywC5BDo@qGdz0fV+oOl{(Up|o{JYa(sE?c)Mp=K zjw!_%%n=N<5H;SL5|VR$#@!3OtIp@lJGFcNl5LsSY>z8uSxwG5Y$P^KSbL%L;VnC9p1*-NI4<3Lpv%60ar7d`_(SY%eF2+(MIiBe}Zr?0gUrfSb5_U=*m!3@fd6E7w%cVlK?V8A;siL(I>nvKNXfu9=| z4eG%EV{>9s)LOJCFnXhsFU!(Jm&C5ja%i|QL4$L}jV$$(5^jflPOxTZ?~Q-+w{Bth z{hF7~znzxGuJ(@qea-&L)%Ryq8XY!G;>u1dI(PH=?ccw@-AjM=zW)CY?gNE)CMNK| z-dW-!88*4AWv-*`x|SfXsB^1!O|zWVdD1hoX{+>~f)3pUIr&nL{iaGn`h3oEl zmvWikd2&v3Jm+~sEWlKC#_V3LmmM;qc2ZWGE@!zNdN^x=$`-Z~H$%#rkV)O4~4}=mE#B zBU$DvQ>R2ko$OkYcPKIFc~RE3Z%WBrYsJ<72p-%%J#tZ{T_E2alh}DmFV@(dKfx}y zE$Gi7|HBa(myW-&(RF;(yz!msGPR`#un+vBP&$4 zo?@BhWp<`YLNM^X@D=x~k4`CVo*%er-kBM5PI6t+eb|tGNn9$AzwMT@U;R_7%J3qLcP4H0DbDCF_wC_9VRc!6uE9x*maywy6f%xy^sn z^v=JsQ7+-SVIs!$OL<}8!kE5It8ekf9$G2&Te&`{g(v;U)?dws6&a646?K@&h8t-{ zZL*GvkzONPA-Ua!UAL3Zt7DpNc=EPS&nFvgw`^+LC^&z`((9@@rqloEh!ifm{Xy;N z+_{=_BIZ8jn^ly4<95Yz$^3`0@5H7sWwqV?KTq=a@1D7Ho#)QJ?y6dqXf?g}HTSHF z-@Se=ZBEmhoTmHln3D3gEwwP|2J^fEl`}QnOV7^~Ydjxg{b4ERe^J$5HJ`_B?>!EF ziLN>;G5Ok|E8myMnZ+-CCuQ=c!;njEk9{QHqx5+UkJhvLu4rdaDGLc8iIO~&+Prbi!_tl+E?z<|s%00+sZ4Z06 zt~qm4*J7bM&0QO%Z$B^O;BH%OCam|~>yOp3?xo%8KWY=LWEb{woXAw#HczAR{cWE0 z-FA`3k3^3tFhl_W2#;KHv9cDB9 zn9Z_yOJ37{rK9PqEhl*zMomjvpp>ye>V=5o`f04Z51r(`l*Mgd7$;-m z-pt?Gyw?5O+N4c<({^vTvi@hj@do`5&JUD|P5AWV`I^7%3R7}+Rc$Zm{M9M6GiKte z5Ytt4LALSJUWM9BXaC#ol3Sx!+FNRWU~B44CR^D`oJS(;#g7M2zp$p+Ee|9l`VR7`I z##%eoHmfPiW=BhI{K|Ft9dWs~d%JFzu26Z(KHIkJbMvPcR#SgDSk1GvrlYG;UgOKcuH*XVjA2oV%9myeJoStY5))%jv?~ zR0rP~R{#EMEcwR|sx^|Af7*SMm4U&T4__lr4W(y-vBb^AHP{)}o(s3({im+2hLcq<3%04$z1ekIr%X4~+UT!uoq%tP$D;UN`$Ol|{;@ya z8QaS9A@%ILx3;x@^FP}>|ML2L|9plsiBlcd-SBmP-p@PPm?u2vuj=31R6arT?H9g^3;w?L)^~|!S#5t1< zTR!vH@?Fn*eP`?0*hexx>0auYp1JS+thXn8V%u)hwy@-*&x0$4Ycj4@tiJQ*RJ3;4 ze-fEG;3PANk!h#yu$LmnYkAn53w`{ZH<;LyZEVMuJx6De)|;m zEl3awHqvV>akJh1$g(Zee&r%r)ic#2p@0}7Z zZrU+f{O==|CI3?Mw6;B-I#;S(<&OIEe?3zibUS1ZN($vp|10?4*4bp^$G*l1A3AhC z|223M(>mGda)mC7SHwbp&kC_)HgZXll?FW>9~bxQbV(>p&=6W0JgMyCWIo0B%KM(n z&ye`QVBovE;mGFS9x3;gXWAY(F^~V%QqxEFe;i+H1Ua6Hm>^Ww{^YjtgybCp3fvqQ zSyh6(+a>DfSsnPop7O2mt1;_tt4r%HTlp9=J)WiSa_6kl^T}r)x_vlxv0;z)muEb$ zX3yX(SjiwRwWD^|u@;G4{~3+$SSH!A-Lv(T%@TJFKcW9u?r&<>*yd7`3<_|*RGldC%;&?rTk@N zXCPr(Sk@VOhbQXQ59SI4igqf9wx|WQA2oS>ZD@D{z6b zg6Y*OYj#Dg%PYP+`GWdl`K~5|UCC3-4ovtmJK(y^?rT@}Zpq79r=2)K<5E-8EXfm7 ztPHqs`m!G0p^(fWE?FhB?&`Ij*|T>av7B*on&0K1&3moi-f221edm)zo6*tY#$ff0 zi$Wi-ExN3ozxIaj>Stn6R`&(1JS5U3a<4xz(LJA0P>RS3sccKLwwt|K^z@@uc}B(D z8pj&PgQ7ksXRv&g{Q9QrxkT9C67jbhiR+WxUQ8E!I=@yg<)x$FF_v`~P49?>{uR zs4tnJ-=4Ty)Q@-8vFMBQTSR~LwWK7Td2BdsBKzTv0{Pm+z#{^UE)zB8PTJBvzl7x{ z!$;fy2U_+sA24~#By?U@Vd8Cfqwq-n}V1=T-{C%->eI9{OX zASSSg+hfVJK7)@&r5!K(Psly^CC*h=y5#PX(iurC$9VS(7MgQkRiAx$?SrX{87sqI z+_k*2-B9>J5Tl++!S|eA&WAfeS;0EZj`5zoH!LeKqGkn$m7Ah>FflOv!`r#nM9B(B zYg$l;;~`!BsUgw9mjeXu-Hn;GM1;w<;m(t5Cyq`$VbR3r5h}AJYHd)(N8M==ho&sK z%y!&;Zeq>1?>`>!e{i?VyL3q;+I`NMCuiQ(?Y{qR-rdiiuYZ?k*ueLh!_CqnKje<> z`e#ZVi>~u{mu`R968FIT`Yh4q&lHwqt+6XOFEDL-_wsY@$JbfbUDa#nJ-2gc)h}Yy%Wo&$0ydLeTu0MG4wm6cXjcUZ35rcG2YJe4z#?z_k--e8N1(v z)~eg&8*CSR8d&l-tDIX1qxI$pE=>*r^l+P8w{XVgn}oLKpE{>ohs^B(-p zQ(fd7@pg8Qs>NpMcN?#75!PA0aO&capPzPQa;m)WSzBA$5p2+!p!U%Fy}xA$SDHXu zwCnMYdRI4V9WuGY(qb7maZ8UvBhP{5Oj$L{9FkWb3}c%6)P$$x3D4=~+&=;{Wd9nR zR*v1FbNUf~rO2_Q{U5(oGOHy?AFFTjv{Os92{K*Moj-qngI{yFx2eaZiyXUBTwPV) z^*3rx*%2+2qH5sg<274S#z4ckqv(nlb9+GFtR-M_B2#ySTJ>^! zm4EdPR(rst7I?ODpZGHtzm4a&_!&1nQNEUON~fmPa!=L6rf0g7#l(y?UQd~P(!)pX zZj|JbHGv-`K>@~gL(Y&&)+Vehg#iI+^@@JyMI)6Jar{?!&ejjtvgS?eBW zpW4eJ#=Cmio%Ew@apyKJ@I5_c6Ti+}Ij(h<^J;x>zG{f_>yNm1FeSWoOXky=Z#Vk+ zvv}^gHEYx98R-=}Z#r#TyJx9p^_tqg*!=DVe`gr3*VMo9O=$9W{}Zm-^ZR?&|M)(u zSgVkK@$aisw)fZ`IM*!I{#>A)Q`7$3enwCg;lD6h{~!|s!&SUhga)pBfn}A~+|X$M zD**!YW_~NVy@|VZ?V=c#iBm-vX@AsX5fy1>jEwEwo)I+d@Upbb>v@KDSx&HgA-VkS zyQ#e^^lks9J*zSPdv4Cl^YQoV84f6&QJSIpUqRz^>2@u_O{$W4*Ha#}tb2b||LSSK ztBxnGr2e>*?5+8Qxo_6F{C(wiR;{ZZhjJ}7DA;@IQkqP(#z}r}@i^s}zsV;)1wQxM zb5>)g=y^{5MLJc++J`FkO}4ANzs}WFPw~W`!ZWef(}Y95tqzUs(_t;I|F2L|wrY~* zp?xCSbJWkW-Yx(BO-M9|Gv~$Thpk2)HZRrI8QUKcIIO#Kzj8DC{5w?yrtLLKbi0Q!#(Z4O~(r3Yk!#g zx9FdH&(0( znljCLMrmIAkF~EuUhIm|v}jJ)dUJdEJ>y`mWrA&!%lhW|e(q^AU`=OyD0Bagqo^a} z)QQJB91Q##?;T=G-hABt!ejsMKa6evIZ7}+xZ3ovLbfsIbiT3O#YTT4xwi+FOU(EC z*01{7!sm!x&wQ52dswvgT`pO27?htJXX>^31cjLV<n0@=x=}DfKIN)y(FejWj9NYhLBJS!-tQbl2zlJV9mW7VPPbnwUMk zn(eIqglidUli%*zal%tS@lwmSpc96%-!?gRn=K8i-LECY1VM; zsyHz5^$Z@q*>cAMEn{;w{1r4iK51>y#+_SB0Rksrx#;xT6bWg(JoDA1%06?<}4l-SYPSb?`_t z+mj{x<(U~6{;=Weq~Km)nTl&_c>3*d;c$tg-;X#bNCtlF+S@SAiRnm2_TGg|ijKh? zj%C2$#-$hsyU(Dt6I)E-o3Ea zk@N7woojZ=X=hsQl`+nKon&3s>po5O$MUl>kKfL|%$vJ!)4o+xr(Fr1xG?!b)q?4& zJXM`43(L=`9NrXl>1dhxv3r}+?wagy+IU9jH>ZC@G5=Lw)3AWv8lR0`yC%xd-+caf zYX321Z6UvX79I>nm!F$v-!kFR)jp=u(=yxH{Z+@d>wTwR9NZ`HEy;ds+2-5Qk3{!Q zxa`IFCFtF0r+2G`zO^WyHsX6&Ai!Se(zNEz9RJj3QlWYqS5MfYYB%c~4q&nmbuA zN* exp^(s)%WaC-^~kFwyk~C)wBJ>wi&jk|IPa5BC%SNBlfgrm2pTqPuzr@*tPvz)@l1k zeO=PRnP0ZXc>e8kB6kyx&#haT6!B}9SoiGNQI#8$bT$ez^eV2OHQPJw)P^m4!jwWR zLf5ag_A6936g|T7XitaVk(phI%=0-F99>_|J2bhC>H3+=CMwBn*PT2b?P~ek`-=6$ zx|W9IfL&RAvVpAgW_BOuRi9$m-=lGy>AcfJpNIyo+$}%enY~_Pa_oYe+!o*W1vqcMG^;{e)%@$Gqm#~W`^}ZP z?NL|7@?}#>jAXZezN=j6X#D9K^X+a;$<-&=47uZO_wjBGxH!jo*;(Fg@BZa0ti5P8 zwInUlV~O_*z1!>mMHb34S+32wxz=x!&J0&hY17oeB*VBO_7fS zJ9{>3qVBHPw<38fEzGlTJqZ=~a5=b}lP`bQx1`3@nkOwkBV;<_E^^JP@Ga5kK4~(& z?%A9SE%hg1%Y>4CPha%1W_xRV&5?sSj=G;^zntZ}Z9QG7VfjDdA7b|%|9n{>UHyRl z@MiN<{VI0*KRt9-pFj7TgZOmDO~#!JzaMxAD3(9w=Xv=2zP9q>Xp0AjYrgo)?D;I+ z_wAeQ;q1kKzI^*^c=*gaAy znB0BMa?bX~bq|>Iiqs1vryHExdp9Sx@Z!9p`5k7KM!TMu`{XM=J$!%0R>RB5p=Y)n zsNuVNLoCnZ$I?c<^XqrApILpogxRl8>h6-WDlgP>)a1i;t0rWxtleps@~T?IHP?CF z?A=pdUt9Ndm+Q_vtUr1!EcusifzTh_GMt0j|vU6y#XU;p;io@)yhnO;1Wb#;pk zLyismg4VfhGqrpgI?@}C%Q4KeVSg|=t)Xt2l)?ed^9=K=K74+z|Ni-67YjH0?^_=} z*SArq`}(QlH^=7r9qeVjFE$^q{qr|U=Py5~A;gz8|C$9W1A{9+zB1VaXG18zs03}% zJMu>6t>HJzg+nF&-%XpT!nJPzr671)_30WT_*xR%Jk2B+_nVK)-}5w@sUBOrMg4O1`N<<{jC%B|&NJi``d#?3iQhb4&h@xIl3Alv=MX4!;6#8kyFH*Qz&7eepoT zT&VlZJtLD_zZspwZ>z9KF8O!xo^(*H%l)P+neUSvb&gop=^WU}^L*YXBa2P$XCsAX zb(ELpM`{ZAo_%;FrZGZA`iPPVdx4#pam#Dos6#@hYR+9KHF9F-@jf*1vW9oaJ%Iz< z8@C85>L|asS#tiswtW?Demu*5*F+rLaER}i&g*GmA&UziH_Y0;@#~CbMtTcV!(z;; zg~ANy)CZSZ2z4mDcZ64I{i{*{rh=6tazl6jLFabDG0qGs8u zC&5uCm0})RstO-@x})iItY>FojkIg%p-`ST42KsLESc4^dx_-)&;KU&eAVJ%789MD*79jYYeE`&>3q&9Qgh$@ytL zr$T48WX$4M7WpSuZFm_Pc=>0r;){7rVOKPswHD8A5%rV3wd3W5?CnN}Pp9v1KObY0 zd*}3mnc-(6wyoM3e(l2RiNaStSW84@M?Mc)^wqfTPKS%6SZ!CB0`Fwcpj0GzLZMS&6++?@pW83b@RhdUD3{sBpE?%HzHQ)K> z<-HcqpFC4|$M-<*j`&l?ze{oWem>+YP`yp8x!0eLlsN(T;mEe%E)NQ=3$EuIKU%N$b~-7jn&>;>)LU>0=N7`b%rwlogId z20k^Hy5+y*s?y=F|H~(>5W90__g_1iW!bFDmjq3}vii%{|9tnJKG}0z@yioU<(-?) z_b^<&>|Ob5!u5Tz^W|>t(_ouiQfVN}x-{g`)jMJS+#WxA(n3;CZ&-Iv;8xL{i7onf z*Z$~Q=lJJ1yM0VSv%$qY-dc(OHLm?^6)#_kvdalEH+}Q=Si)w-x!_`0N@f)Mi#IMO z?!KLQs8Z2?yZLA1CTpjA^EVU--FCj08B@R!zFq!k(XL}%>6cHL9=|Z_V@>x}$?Hpt zj_nPz>+9}QC=v-Y?_0F_prB2cUYq9>=XvrQ9twP!@Z7?z#9J{_YIa|;_jdg!ufLs7 zzF9u&Y0vVN?&m6;|7CuWJDz2!{PC&TzBLcOoKE&gbDF2@Q5&C_GPT35PwY$5gm-M) zFI35IOJchHL3!Kb@LR>(%(!0)^F`NO?Pomrfb){HLUv85?An;5Qrk;YI%-ci921EA z@R*Iq`j4ISZnsIZ6zi61-VvF!tJZA4-NAhK6Y4*5%OCAuDzUo|)vA8BM0W%v)w0lWn@@sguXI zgD-jJT7ggJyvvFoWZyacQ|yfQ<{9r7a6S3oy>-Fl9dZ3K>kb?)$*vKJt(f<&?Ec03 z@`8u+Z?xEF{8c~s!DoppTbkC)87otg*uGoNoN@54pIX`n2Dv3ZAFev5%r`&wzM|z` z)yY?^=T7;@$rs;Bo0By^yX@?$gX%1edF=;`MQ>gfDOWwv&>*zXE=T-8bk1hc2eG0L zVk_JjzCVt%d451b)pFf^OYURbOl{o>b2ty3zA6y*P-u;qfY@y{Z2>D;j=DWf&V7&D zJ-a`P&VKr1>#p6^atz*rM~xU4+%=136j)s#aQL;~r=$vJmGk}PAH&Wr*%KQcbT9o! z3Eu+ODN~-+`<#;Y3$j~~wf^W?_tIix{wb_S&K|0N^6atYogGIing7-)rPzypldxNU zVBd#d|Jee(**P9%oMt`9#=!7W0J(}>hIY({1WFa@kqFr-3>h-(4UI0o9U^cq_H~iU z@y<6lwq#~L-cqJ*%<4b8ZMIXJne*%|d7Nd@5(^rX=38iW$)?Plxu9fF3JuWtW5H}6f9(7O35ZX4pmJ*>2wK2KY>T=i+-o9~A_w!J^y{BOgZ zoq?RZ`{h;FOuIB|B_H3+*}@XXWQ@HGE}4Jl;$RUx_4mN{7{2p2K00*&xYRynTHNKU zKkg+nEfTH%AfqIrR@!%OCf{_yHJ-Qhx6e21;|r~;a(eoZ^Jux0Tbl4eY0N>r8(zcVkW4)6E+Urk&qd zBX>{sw0Oimj&9C8$!S8eYVl3pHGdW^xWaPj;{5jgjebcXTRbIMuPzZ(TQ%e1wv+@r z=0lrK%s9I+GpLMh>JCM}j{9&J2tGtV=4*UaY5T@~ew9dJWAZu4l6?-RlCN?6W%^V7!#d{3$FI{0Cf1%}(-qY?Z0+)5`osB$l~*KW zwRlBrHTtQ&%`se3bDm`1O35#AIvDwLUpQic$#De3jSGN7Q_O#D2zTWv(!RJ|P z#9wWf`7XQcpQ!7veVLj+m!DhYXfNU+tyj13-;qAOfA7PCm$*i+kdwY;@pr%0q`NiX zK|ii`v({#228IuK#~IsATH4Z7T^bGhga`X;zSP zv+sgUiN6gv4@zvE`^IllruydFH)meD!fq!}vxsS*fS`-(Mum_&CN4|%{rutoV}8eT z>&}L{3jj6^Y%P^IXg}T;VC7b>BXdM`zDU_I2Gp z@y;&Me2v1aX#%k?u3hNV<+{~1ExeRh-DBF$S#ORh)=J19GHWc{-8Ap@FPW>_Axqp3 zetUgfR62gvTh2R;vRv;td}Pm;opn_<53m@BDM7&x)eEMN5`D-(yi)V*19(%`s+u$Qo7F+VGeD66x(m<}E%)YHutS z{CJluN;WTk!Gm+s8X9?%EZVDGcGju+?c`1ibxe;qxzU3w_1LUSvcZ2|AKljJ?Rx%F zY;ws#jd?4py8rEKWr=XEK2-WgOKx$%zg5p?g}rNvJlWgV|88;9_bdyIuNN-9S={6h z^_;gn<)kTx>9liadQ=KI4okf|{8vYJeUOsGB+v6b9D5H-OiWnZcza4-Wm|lCvZqal z@~2miP3*+iS^4N(mD{Krr*A~a8bpcOEb3~ z*X(?3p;-N~^O)kaQc>Y2sgmFJ?0(>UUi~A-G83Oiflgk}jit4{UL{89*ey-CeT-{e zU-dT?`H-4f?dLY9pMI`>x^-(?2E7xC6<`j3i za9(qVw#vz`vyOe^`|oJd0DXM?1Lw}f?tO>Tii{^zJ8C} zvqt>B@Q(Y>75zeAyf;63fou7W^N%cdw$&|oFiqhPXOh9Qiqtu#22{}cTaIM{hZ?f2e)M|x);WqoH|8+YKm$*(;czPDCHqy?^5`zW-G<;iImu~*9< ziJp2p?fOwU^$O*w+g(LJ&pCKw|DpA!e}vX+y|}v~Q)}t%)arAagG=8A|JAC~X8pRQ z?fQq^Zts@u(286Br8xEYg{2!ydy?_0k_aB$FnSZ$EyDY8wfAf;Q%nIrH zClja6I}yFp^2dRI;OdxTsV_6NbK19^Z~X4>p?X}sseeVvy=}bdS1zp%-Kl-+nnB3N zc}*_OYS#bV8V`xF-?-zfA*R~iwSWEYjZK$=KAykf_lNm^)1R#yau#(oD*Oz2vf>1L z;y0m5+Bakz%S7j1K7C_X6aVLjLS^fcu9tj?{StRD_VkHs8_HRk1^2iv=V)#$6@Gf- z;D*^3de8pgm_Nxe;LYK>e`+^$`L76{yWst1cUa+sOuMQQ|9D>c)ejqOO>MqZ1z&Lf zVUS}VVS4p_1p4U!&urkTZaYo zB^hk%x|ew=`Qug3V;)7U&z0xI9(z9hwhoW-fwjJP0wwIo$~Rky7%$N ztMzfL{auq|sPv0#UzEGaA#=e_&5n6J0sr&k4sKunH0xvXn=i#B?9+N>-hP$gtYP=R z{y6r5M%dvcA9?osWdC@m`e#Z?-MQBI6aW6DiLUoe|2?g6W1IKQhPPVx=AF}99Cp~c zuFyREtysnluVuE`1w7^jJeU9bpOOE%k!ybW-_oD_S<>P64~YI@l`LCd_Gjzj`D>TR z*1vx7AAA7Tj2dB!wQLLwDZI!nSoDMHPzQN1RtLe)yeUpC2}&)6@dv7tM*D%GcW9IY4BVSbo-WPQA6WNpE+kT7JRS%?-c)w`_}?J#+9Nk zg0H{ofAz|4@&q=c4Y%5hpPiXg`|aPi@_l=suisyPn@QlHKhM7@FC!xD*1epwP&+kk z`cB)>2h*~$mj%jbrG>_(S|rYJyR)Qp((S%fpAS)^E!9K^Ii5-4faqUdI?!3!g5C0x)y<2+r!W(nR zJT`t~8-pO*eg7icmIWHS-<0dmX7jiB$+P`ZmfMs2-`7_jy7)CS*qGyKgyv~(r3o#I zd3q22iS96P zq|!->8|1gN#M%3GCd_^6a$HtfTghe0nss8`LgzTVSVOt)h8pZUucalhzd`)(i^Gv# z_v5Z~@l5AWyuw`>a%cO618cG}o7Qt>p3Q3fCaK3_d*hps$NPJ?r3}6I%qZzG+3UeE zNkvvUdD^A4GaEFM_17CpD)Rie*zf42E+PCg_P~VsKe&xcuKzxNR=`}yW2&!lXT+|! zIiZVJ^h_^Y%6fQ*S9z?E(>vQ4>YVEhWj5}ZwK<&Yh^xYl_`*45TvqaG`PT(A*XqhXDm|Ej86BBeZFqrp(wle zH=KALil|S^H{39(XNpGxlS|p=pbN4=cSQYLW#08QoPYE;Or>!4!#{BgI&FPdF7vTZ z$k3fVi8EZK=7fT;-`yFV{eg`#z1QDN{NG==omKv!a9g(=!``Tl75{tW%O@-iF_kiW zvr2Ct9q6liLu)ny)xzQ*60PMx7RHb-|*(RP)Xb5ya4U>t&?r?Z|zYtZ+^G_ zm)8=nRTI7Si+-)C5UIN>-nRN{+W&`hda5stKMVBo9kR!6KAIy zz7wx>T~x9#Zo=y~DQg}XKm1&LDQMLn_esfZ2iFEVYN?A^*+_|%{-`~tTI2lU^@-^p zWED#eXQ_4e-I~2_Ef=fT4@H56>x#N=ect{<&VG^M1-&R!2mh>l?Tx7&{xUPMdq;qEEGVlBbk;=Uc61c3!MElvvJlUKcC9^ZbhXb_1vLfjQR` ze8V>SH6_n8Fw&ardP_Ls{Eqiqymg+{FJ-+wxu`L1;ylmCa%;SruD{q&@K(6QvR&}$ z>Z13`i{9-%C|B`2^5_1#u#cHX{d0xlZPs`%x>(y45Wh=r-y%hqYmR&Wymoj3E(w2M zv;Dq;g@NG=C%%$U2BixS1Uk6WIj6L^B((_ApI;kQnIQ61v_5~+&Ezs6Cl3XgMh@v4 zIZTR-jzT={LJAWm%xIB5nrf`bJo|1cKg;=ymx-~YY$UiSXZ&qwOzYa|$EeQ;wbRhZMk z_QlsNz+L10_4b~KY6HRU*B4VaHrT&#-{9JNL9ylM-9P2Y3Nk#e8g~ftnf9qY`q%8Z zcjIySOxFUI7P&)7*%x^)uK0iC*pUw#A9hSC0!E zKVEW{32uFBU1e7GP5#B*k_G!Wuiw48_ArCfM-K}zb;j*0trp0uo@Jf9|5};+H~G)z zY=0Lz_c9kg^T?L=+`-o4tXOX&s5<{`cJBnid}ocX%NAUDt8)DIVP_SCwYNE5-WFcW z{`G~zXU4=d1)EzEsSC4wcxNzcnOse$gP_+6o5&)wjRtwze+ zi4(cR88xqX9*w-SnOE2CL)3%i+B3}dXz4Us1x;GT^Fm1YD~Gp^&ytg~#ip&gz;Ie% zm73|cAIZvpr`+K@@|0t9kf8d!*&JuJOb&9bw%`fyO`E*uh85pSu4TPfwbnFwS2XR8 zkleB&gl(}*N~C&Yq7bx#MYN@0iV_N?4v#* zKg>k8IhlXmt0K#uv;0TTFJ+S*J{#^#;rPh7c%`W=Uv0rm>8geEEko45XCzz5T=}bb z@1SgBfrd+MPt&AWR=yi2Cq+)bt+CY6@J-oay&IFm3_iV>krwIkZRLjfKckbjZDr@R zb!pMsdoyyyZSUroZsB!4(rQPX*=3eA>bWH+&iOt$=V|TDTq&N$bxsUznSV|xyloJa zihZ1@*cIS7aRP%(n=+4rhg^rnSHU*BH-!!zk8WSSy>apG%ZJ^LZ&|)XqH(bv2ebAw z3HIBfU)`0~?3jM?#_V4gT{QA~UiJQUOHetf$j3Q#L0{m_Xiv)r7Om=X@3@Yd?wD1& zS-60g_o7g6>>`2n!3P^&pJe}k;X;|+q41Qcdh2f*?wI)WuiC0eb0t>VpV&PA z>yD|thszJPhyD<#X^ANO@M4AI7x%-3dp}n=^czI+{gAMn@JCX3qJ)j$#pUN3GdUAK zUkNVq^N_m3`+Y;?@5v&I9?w(LQvJK*;B}kTTt<#dtEJv0cg$Bi-uA^|i@8b~=jnMj zr46+g?KN0_DZs>nncv{^<0o9k`zK!7=W#32_-wgIlAW+yGy6m1Gyl>n9cHhbC&A4t zC%DD#{JR|o#5L^PI*z+HA3t(gUoi99wvR1r{oEVZ3ovD$>UijKcJC&MQ-LNGLi>*8 z)Z5nH@C=h@HCu7K^INde)%%MQt(F}xZ)vL1xt4jYKSyigd7o%5&OLj+&1-va$B%nwR`T5~P_c?GL3HkL4T`+3VgdcJ#F?!jl3260XG z2HF3Yh%%PXV=3A9f6~eHur;A!MH}ZvF4aF=FER1|C7+`mC9fAce-wP3neHV#lP%+B zQwoEq@!2cErdvKnByW6K8=m1|pJ!|3^EPZ+AiLE}MsdmB7UkwTA(I6{{M?S-qnSS#!<}+CSK%Au9)$A z?L()7X@X0S*DOd}D-fr%$c4MuHp{EJwPbF?L;oa`)XWI|5UC?BR~K(SJgu`cD|%t! zE<|N2ePwm@Z?#MT>s^Onxt`elck+UkdCZ-N$bdU{)DwEVd!%)HB9YQe+v&+bND zKWZjpq*Jvd(L67G)n1c*6L)3h75+N3@w0v-yYR#X_wLNv_`**9Y>6xH@4^X^_YQXJ zx^tZ5<1ROxa#u1vaI(gwT%C8RUcYZl)Z&!=@WTD3-|IieclvDP$kAJIggK|g`g~%i z#oyA|%N?2ay|ekSvR(U(kKv7(GZnWn`^m9$&&U)u_~SE!)B3>47WW&4xv{UDx4q)s zJlUefcIVv7qMK}vC`*JSiUz6PPP#MW>XhuGp(ahSX1+aL-x4f+=J72#d+X53<8thA$*eQv#MtB954&6o zyW41fL1?qkvqste$5$(K-yC{f$nz%PT$d1oNSNs#FILCvMP=(wtoXth1_=9;IW=?Q)B86zG@w@3Z@@#3<_{Hb3eA1`LlKJPI5 zy-v=Z(piVkE_grFg`q(-pLOPy1i_1I+pS(aId5UH#hIO9zqjh<&Ue=iR!#2ezvJjO zCff^UC;#*6b$Iwk*FT4?PWH@#i`u zTF|gh@{sHk=Fdx)?z>Yj#wTj}y+b*{ldw(=j~)it}H zbKA0FD%0lN^oLuQ8!a;GpA_hO<8$X-rU*_`8!heDsVN~*4@wJ+pKoQ0^DpPh)NMDv z5G(jnuhIOF?w#$23y!4hnvpW2X61*if6hLCAbT?|?Be$B3$1>pv60yu-`8bWPgRWa zY{*;rVvf&6u3vTPdrphQri)Fu9bV6OZ}EZ1ivN30DsZ%Jni5o?KDX@f+&h^m?(fpL zg5RDy^~KG&_#5M|Qt`mEA(HL=X+c*X2!A>2^u6%c9p2?Wmvq%KcdxILv734_-1}j+ z|EI0N6|DC!pKS6xyz_s^Jd1RV2%$N}KR;DY$clYX5_E7w{cV|jzbYSp*dVjQtwW_z zMonP%Zy8?6HJjun23MQJXZ1$}{7!m!MQ(EN@1lKIJH}A@K6bQOc z7ycLguqD35K3#a;_7a(YVH}C3oH>JiV3>?X6-Prdh%Fi6>ER-i)q$zlh!|8S~O=W<9(l95~ zB1QDvG^cagESp-K!yeZ}GU%VOZBVso;MQGH$+bL8d-*n%_^deYtbK=8UOl~1_w4jr zM%_}GT6^4P6}G07Em)XvReQGVk$??R_wGl1dYiwtXpZ;iqnVQ*uQ@NC-t4M$i`(C& zo%PAGLyG2)#b$25{qE%D+j(z1A8}-r-ES&$TaA*tX5; z@pBA!1h#GdUvkc3d%=Fa}_Ovz+ z- z%VOuO`HzF1P19D*RBui7`uayGB<2*8YoVV^^5i$JdZ~(uOXiDB)10TO8nOCOlFqaA zHoc?Kt9JzR@y+#arMJ5Pelu4}o*F5GeX z9_x_>!s#ieuUW|6mJljQ_!OJDe6Rb7-D*)CCHiyLygd9wx?c3g3!QiB$CzjQ%S&oC z^60U(jyZKM>S*FVzn41qd>2(0JnB8=b;n!quKA0gZ^B0z|E}2lgG*1Ldafzo8;!8W zzX6?{o78T2T~AK+{m%}Xxc!p7xN$Kr14DxzzRsQ|N@ovg^8@NuYDmwoHY7U#a)iLV znLlnE&kMeEOmq67qD%&H-Ik`XYEMlOjQ|Z1jyG8svnEcSvqW#*-D7^?zmIu!q|5y< zwNW^tY|r)f#}VFzu6ru}oc)<*^SSa|&96W2Y#GiRwq#i@_`O=URGUMz>9<1O>aDt0 zKVMepo5byTn!lFi@cZSjIYjeI58wZoy->b1yLXGY%iUjhjP35dW7l*#csarN-b0_u z_aCfFdA>>b+vKlH4O}*Sd3{#RT3B>d*sQ%$^Op6hYQ=iJ`|Y`^>Wq(5z5CkIZ&x;4 zyYoOJwt1_hyQ7rxb&f@TJprGdg_Zu?x^?ZEnO07v{n?52t6Z1e>|G%JGjsp)O?kVz zqGx~nbfE3ly7#AlXFMrMtiRp;b?b3_j{i2nsBnpKKJ;4mp@XQxOnY@zkVsV zis%fw8fq(>Yqq##5{X%e+j&v>xqnKFVog^i#&B!A)(M@8riRd?DI_mAg9DHH(Tw1 zlbV@KnR6TYeeAn!>_xv=+??b4JVxuo8_(D7wH0!o6N@vCOvaqF zt75(nN4;9mbVWF zwQFPF?EJpZXVT2y3C}p?pPW5Y`+VO2{mFn6D@T=~^ zzE2#&`s@3)hbyGCIZu?R-O|43%f#sYS9c$dmixeV`;_g*4*MH!O5eQdFUUtbczI3E zzdm*C(tB4G{tH@ih5;+|4O6BdNi>d-9EuCrt z%uje*IJJ7(JJS|NIM#W!X&B2NJ;29yOhxpLfn;LWmeWSH+tuIXPJH0wIwPlP@|}|h ztfE_sx@snWtk9^KT$nbaKRtn+Ie8kFaiZCwkjr|z{9ny3KEbBW==CCX!P($My$f4T zO+V|wekPr9{?bEC5i1&b-{dsuTvKrNnVQAYZn*yEmyAmb#HOt%c^)&XZDU#elub^x zht{l45Vm%CnYh;cQM&Nu8FSA?c$`U63XW2n{j1VurNsW#R(~1i%M~8r5|z49BI7o9 zNoK2H{85j!X`5>;rG1orJ@YNru-860{jbu+=X=+tDozp2DDKx1x2jHMex7sT2z%^t zqvLMNN`((`T-3iL+4Ix-%coTr^lW$SGQQzqFSCKwF`Msg7SC1IS(`>Vz}6Q4D|IVSU%F3O)WFW~gHe}1J5k>^FHUq8U_y=j5$wr@(|txk-0 zPX`rb2G^%7o;!j0!odliEaDc5YaDHjBbfzmzcI57yQ|;3TUGML6?Vgi&9W(*bARPU z33wb*SamSZadUU9M#-Cf&vaFPm31u)ns?E-Z0dz2#jFumHY`2Jc~_F*@V#^{@kt?` zXS^TlnbaIxcc_1_@nf!iQ7U)WCG5M+9lo+aHO80m2)pph55aTFT9T^Xt`oD>S$1&i z5%YhOn{GKy%eW(}mhLm*mV)L2fpfN{&-^_nvzZ_8%_-b5^Uad?i}v`{aHZ&I*rZ1@ zEzR7+pO+}H{Zc?}mG=@w4`*3z*;N~|oNq)NT)mdJxIv8f+hJLre|}CQzT;LGQHPI#K~13U(haCHP7Ob$zoUT#I!6MA>N(OOeT7nyyg6%l>9@fmEG{l z1J#E=^*>8WeseF`{Avo1>G7zRNwIN_>y4(Y7v!?+$`av?6J7R;O|a9*dg`(i^>d5g zeYw$QTrIorXhzL!|LKx%%$I+)oh8f2rQ)@`D68~SMAW<}!P$%Kc<(jHZ}R%?9+GTe z)hlTKQFF@2htpPW;W^*?QSHx^g@1c=BXT~dEckpVUTwa_#p7G7d zNh0#re*YvHJ)hq8m2ADI+*>sg*h1&AL_N;WeHwP*!m*tT=gvvJbK!z!OYEVvH9UP1 zmm-~_gtLCk)zFqKobbXZLMQrk>9+IlW4BF<+`8z@X$zKO$)I@(du+Be$5@&#bA#jY)(?HRupKa{v= zF)fI(c#DSl2A7E9Jqg=yJilD9C_`(h*2&u^gRbnk^6mephr1`*3LZGQB)59G{j^_~ zSzcL-o{qe~d1XHP_u|%2wH(a@FD;r^&h|{p=>dmLlLWFOc^%!pP7pMW(Z66GpkuzhbJ^$C zJ8nuMPkCp%9v0iT`sc;>M|h zjObi(3#nhOm&0bA7tFsP{whILgN;RT)e57``1(mtW(&<+^0Cd}%$5A~XWW%PEyg>b3(yC1D*>{h2A6j*Ho>-i?NmPsvo07`e zmEM;sBr4cAGAb0~)*Oo2(~{4v!O7>9lJVYX)nmTZVg~ghVTWu*63@S#HOXi5|1-0X z2^sT#e-M|pSzADD$+9Ntas8;}YMw1E&6?oJ zVE)LL=RsPO$F5gRj|49Wt#%Dx^!sr7oxh%sZ?)&oX z!CHbNG>k(xy>0@2VQN_nzBUb4BwvlV$*;4 zw_IFa;qveRyWp{<83~QL?A6fp9!;X9Af(?hI8BU}gxBPQC@lRP( zx1DXnn-lHpr2g)jx5Vhh=2NyWHFyfeWm|2&7fL*P*SPsbzRcgLm+p)Fn(bF#{9^s3 zS|+ia86hPyVzNu~B;6MttL2%t&e|zT=6Z*y5*^H`*>R zD_+|+SMRt&)r=B_=c&(ssa-H-Tg@{i@XN`l+%@HkURdbe%Ir9|)c>f!-AW6)C`GGj z{RJxoI{GiL?f$_raqeY_tYcCgS##bmWVxZV>z&!;{RWYBDzVvTpXhr{lXtw{{ZPoc zPOdO*&D(bqpPW+We601xT(7z&rfkQQKSv9U5|SsW&TpROa>8lXEfu$VzEwiKc2kZ& z4{$EwQQQzV-TM2cJGRFkep2RqR9RB$;rT>$QozesJ;%fAFDiDb%ywZHW|Qw!b}9lF2W?+5P>N27;}G;Kd5YquCbZi$`qVfF{%&N8Fp=j~^g zCH8H9kh}9+{K;l*+qdseFn{0vlI`8@G_ikwUvBtNtycN2|Jvs}?4KI{Ti$J~*SP-q zddb(C6rr#GH^_aJpS|MG^wO#G5A0v6^*`P9>2e>{PyapHj-0u|`$N-M;ntHY7ZT^r z3ik8$ogOufb8F7)D{LoMK3(FS+hOBoaKET0f$^Kz^}pfmnzj~yHm~Kg5q-e%vgUAL z<~eg4?{&*9Vr`S3R2^Hodijwfg2HR8lNNQJGJ36b_tGl9r^z_Kryn&wt2w`nT~N&AV5V z!e{M{+9RNE^d*Ny`fqao!Hit#c{iJyzQHD|6={N0e`(#82VdZ%x+X{i2C z#JqCumyPUeXE|tX`1)XK(Hd4Ky}qSKH*T?*CU*Q1$7`*9-?A!9ZX8W?=rdBWmo5%f^>^I*-s0d=ktbSN_XC+aMR!d}+<9t2 z<8!quc88YsoC)2;KF22Dv&Qryu}*#)(S_fVrW>h+pY*k3Sy=rz#*6c-WW_Vbx56=} zj>M??7af_nc#7{ix9we%FC62yia8~+ebUwUTMoa}X}8rcf8V-!;(oKRhRa1$UT$X7 zOf`-<A{fy3FwU~cE ze8-g5Jk`Z|9v0oM?LRY;zjx2wmUV;MG-J<%Y4ZcEd#iLBb3%@+UHUWPb;%rdpO;2& zzB9!zUYq=&X!<6nz$^QvtIYW})#2A`!`1#-qFN8pB`>8EuLBT zKG`DI?%tK}N}ZRldFkd_eBhl>v$Ss3j?HH>Z*x_5Z%_QAIJ9Rv1- zvho{tLymG%dAZHwFdqYhqXoWEE7a3#^5NHr>LU*SgY=5$hDQ1e2a4D|KXYkjqtilh z0T#yvIc{2l(>e4SggCrAMUzb2+KY1Au4umGal3D#X!pCi<^%hW#XZl@@obs4Yu}e| zZ|+Avzx`c5o-u(hO<+#a*BxP#R>!$2rEs}#de#%&e*>na=022*zU8|9&F+VVz0NbWUVN%<*(mnz+IiN!f792-yIu`d zId|#vWF6nH{;k)p&R@1>@}|FbucvwK?>av-EagO@HM8WqU!syM{ge89ww+j2+VAtO zIK@-3SW1*?a9B4#x9eu z;TM!on>a_W{AFj}&Zux^3rpmxeJ8I!X%v6*dKb?bEv*kWi#QIKPJMNg>)GquGnxwT zoqJa4agfC+Jf`Wyb%FYC*M0dd2Wd5_z`e3Q3y)pBuCTiC^4x{n ze57v6dX+C&=k+#0{ne@uW~EyvONLuG-@IqS#a?}(<5Yg|`~}|^?2U< z*2hfxo6auqb#D3+`z+#^K~0n3pUMW7*mJ2FB|5#kd>>m`mIk(|Ewk3J&b&Oup}2kH z>FZ`^V-$_$jN5-sE1cSVZjn^-!l%V=w5!i9vvgefG;6C;to^kedL?4#Hh*dJHD}GN zvDJB8mNNTIQ|g_>BF4#21MfZ4eXRHS-NIY$j^B$Vb$=_QU!L?}lM8o-r1qK*0+Vkm zZuz})PU;%By4m>?Q`Z>Q%T7*K)i}RO`pm5(FTA*J-3ntAIo0HA-BWZjJ)%EyX~#)d z(Oc7+ww~T&YqNKI&dGK2+^Q~W_I28xu8_NZOrGgFg7I+WeR6rSlLSHBe&357O;lZ9l?_d31pU)xTWTb4RykbRS z%-IF6Es_PEu4kHP(4z6^jDcdzhpeJ}pS+TS^1c$j^m8`n>X_}1+r7EDx&QaVZ#kv+ z{Wt_-?oJO`^(y>o=+&!NYybb9zyE);fRS4rgV4#}y-_D;7N6#uaQ;)>%ca6kR{5=! z*s(8sf1JMPX;+26`F9>vUi&&{Pq6NybL$x;y;I{w;>9mNU9^4jId;?Qw@;?1+RZJG zOLgndZah3?^IJzM4@B;nvk2kMsgk?@)E?*T`XZbc<%Qc_-M9KCF zJImGT^ItrD{MqE*l{*Z@K4qsr8cqlf+AsJ+x^!pp-PH@j=4Dr?Zd{#wspO|}gI2(G z|L0PUTLl%)Xogu8pU&M^;MK?>AG>Ln@EVJ&pY$eh$C|5_%wNs*e8$~d|54)3(Z>>Cv2}e zaWZRhR9Rk%3oBput|zzmGF@7`f@y-ws_e_rOcTY=+LY^D`oCHB$)cq4#yb`VpR<48 z=YGTgXWdQBDu*psUhLu$-p+VpDZj{<3w9@uEVR z?<;5Q3)uGWrs?ws`+q%OFmdl=mP%78wy3Ay<|xQ=6*k(roT+|$nD>bfciZv0RU7xd zy(05w`6Ds4MrQVlb2l@bc**x;p(f*zlD)lx&qXI}UOCyOYs!jl)2)Bp1e(}`@BZan z(WbG+(c+C$g;J@GW}`_obKv@4TOGpkpDq^$tXH(j@!IInMe z;#`uA<&!Npr;7G)H9j=j#?tbrD42!+87H({Mc2w+fZYz7ST=`P7SiNhn z{baxIe-eHvf@AwUyQL@V-u-Xp?v1q(`^Nk<=Sak3DUxrs!^JyS{x_)scMf zJ$X!(KJ$bW&VSIIaLq7vk#f3{d#7S$x4@E~8@ZJ%Kjnm`eu}$vecF5{JGWoAYh->l z^4T@kY-;|`^YeWFhvn8k9G{$*xj0`ne#=k&iT3VoxBeJ>dVZ2s^7>J}I~<>;Tm3kF zB3@CmQ~K?M`qn>J`TqHzh*!R@s<-w-ytdV{7rd_dlkB~#x@(kQ=6+hcWD;`H+U#NU(rw0>(E>Avhal^k}_(p6sLSHQ(eS8+;1(XMW6iGx>5t-;R0GJt_$T zx#wpvPQF(%+xBqt%w>WXxe^V=Dh+JswXnmcdRjJ$F`i?!=< z^UUp`Atx5Rv%d1z?OlV6tLKR$FMjN`yH$UEnQ)=O^o@tknWdH5H`#1nrcx#KM8x|L z&$N_l%ZdwEJv@47P5YDiJ1*F)$eiBNDiIs=sx_y@uB)ld_~NpiI$|EHW`5m!WWB?2 z7T*g8S<}`{`u_ea>jo#wwx1D7^BZRD@QmobxZM8Bp{f?UZl==O+5dL(OaB(UBVwH4ETj`Jd&MPwl)ff4&Z&x&y(x~KRFHaS^mrz*0@nznes8_4f zkHtHM{4Ps9wmv9n>(BB@?{`d_d6Y4Aj??RReuoseN)=5OJQkd=L+N3RDl^1B^+{iS~Jobfv zo}ZLG@7n2Ni`bpN%$#9&?mX8Q@$W0!v|>KUG`)BDeYfq|;X{9Hw#0X2)H~jYml2w# zJj3&chRcgo#=T7KaZ5S;TTk~M+fw&W^j6H#zIk8DE< ze93W(4SA8*7RG))d&=FQP2YCQrfzS26K(Os#(}-?;e)<_lm&k|n?E@!?$>@IT=2!2 z`|5Wi&fE$2#XEi0@h0&F7iMj@pL?OiZN@pLAH40lI(KXZHnkVL2{uVgF_N^?|CZ%= zrNGcn`LXHpS0Wot)g>PHUzqSo)#9J(yah9Tq&LjD&?cZz$j zgRk`RzxU)8$??C~b0FzOO0T%VzU2vh-I6vF9r({YJ73}GdnrfG(f*T@h4<#4iJRN= z7o3%^u2=q(UiCpp^1=`QGf6d*#p~ujQoW$|`SOlWV&zqu`U|d_MLm2JT+_~Vm*Fwr z*G4rGl?5{NM(!HSNDoynXwY|FhN$WyeBwMq`b& zt(Um>@acRw+&ry8QS2dat3CVPs~1%LZs&d%57`^IDziiDs!HB1yRHRQd@nBwGCVNLEkZi5Y1u~qS!GRTrGZRO4heOfab`0XKdE)r za^l=y_PU=kBln0dZ0Bj4cWE|Ty?fXPPl1E&2es^Y=g;_751#FsU#A@+$HBm$DTJ?` zk9yEIc(%(KvJ_bbz9AzSavDE;KSoF__$+?!OlCtbM~xHh^BI3HJhDnd$YnuOm^R1T z4+o;9lv*-3&owE$aQ_MOK8454k3nbg=S5j>)#R~D*}l8_`<>!wpvC67s~7S$f#yn5QUQ~hgxZF@f@L-Cz))qSTb zj~dCexfE@mn_V0qULakba))WEZPAi6SI%7;lV&|>suSu}2{Jl1F)cyua6*UHR*$be ze-&)b8{B#jC6dhiIsdjiM_TKJjq^@v&p7Hl^UUuKE3ts0)n*67cc!nEof+mU;e1FY za`#phe~-4!;Z_?wHr_hwx=^-lXTc5DcbYSfHx(?_d(s}bGU2M!`e?3cQib{wzii^Z z#N56qem6JYEqQvLZ~0=GuHX$#a}Lya-qSIYTR&;}za5r=A>FGcyvQ@-T)*;Q;_->8 z)}Op8ybn3mPW;e!g1skaov3b=y6ErC6>36ek0XoH_N#xIoDAQiN4;*U`@=N1{$9FL_uwA!nuaxljKetx4G0 zd08*%t%Zt`&>5wxRSX}y>}Gybn|OS7#W(OFHeb!nx4&azU?{}9u~!?V*`ANB%Z$9s zcWv0uaOqT$z0OW;2d;!=MF+3EyMkA-W9?S1D@z@gY*Y*pny_Q5qU6ghiE3$?D;Hg> zjn59f^L>~8Uq;vSAGg?7_SJXQw;ca{mdAP4617DIM)%J=seOO9{C)NN)AIFyKS?*p zT4*n{OcN>A{2Hda*ek+xQfTqXrnDD7SASlaqNaK9^17E%3uB#j+fOoDw!1hh>U6nu zU+niw)Ascha=v3!_Vw3Rp0&=c`|{*0y?Jv)_DSF7Zr{46<*eylkBnZ49=myGGLzQ7 zb(j~#{ZTmb9`DsQA)q7)SFB`W&Rz4+y-g{Rm6=~+ z*E!u9b;SqkpVgkcJSlg6llO*AtlAsA4rNUatah#sJiX-VoSv|tJ?;TDlUb(kPmVd5 z8gaSjYJ~TxN1eJ-`@`VkHc5;S$WoSuE zo7i&tkL=V;jTCAo5O*1{=bVlIzA&Iwx}mA&fXL7R1lzKP{D$2Uz1 zp8PVpTyU`v>!0`|Rw#<)TixLkD?IOpyzpb)UAWhwn%VbqmX`S<(I$!gR6;7bO1*~jeOOn)G{;%a87zJbRK zWhWc&8$nxpPjM|4XLtF1M=vp3^z)}4kvFbBH;&Joz3lcSleWcbE6XQ1d<+uM=q^ut zCUQsn-Ytf^kN(|16?v_lZ-3WU8T~ZlSALT0H#^wNr);0MH_}mP@>z?PDI0FBbl58Q zKKbwzb-l+MTD-L*rWdxv23-*s)p~u_bxII(+vl1$~k=ls{ak-!)_5CS&n!YmJj5w@8NaaD7kEa5^Oa=;efP?&K=B zv(KH&HouGZa}3Xq_tz2W_cOS#c+dG2zd{~vxVgjUccINi9$x=VYJIB4u1TK?wjB>y zX7SCQaYe?)g#mLKIG$T_zSgSNwmo)J|1s+&V=>;%d9@R}B__U|{$Z22{v**bJ^A5?X>Zvya)>??v^cXKXv0tEh^(tf5_Ybub zKbdx`r2S>P6~>%Vtu^_~=YT)&n!>j3nEtW3+IZjbeu4V69}cmaznJYm_m9UT<$XB~ z?cx1D7j>pJB-L`xJpW*J+LjyApGzJ+x+VG|=D<>33BG?_S3Bo#Sa;RDpLgz`EqPB; z<)XzeTx@v2v`gjB1D2gf?$pf5ORw;nZ_u2(N#@qUBX_U9Fk@sX6ZY}47 zUEK^XCVudhWN2QviKU?aE<@GDaNz^J;*15}s?5PWr85||>|%W5x}Wz!WU%{!<7@(N zqgV`fa6AZ$yH)45M8+pP)L=5tY^9@_!afIz-Zi`|Jn$t!BJdqco-Xuj2~;K5fIZ(wIuw1#!|V zrmK7B-c)-vJ$KH+cMEcaOHWT*;k<44H1*qWd91x|CYh%0-tDxl`ou!k*Y0+wV}$1( zKKR+fqw{#?G@weN~ARN3x3_SwR!{Vk(>etp}=y0~e_I3&>aDYl`CfF_ednQ=_Mi4{D}?vNubSF2 zIjAZ9Y;)S$GuI6H8@E9CWNaek%`&!u|li4EowA88zUpEFtHh->I<#IDYv~xAX zYvmT(KcaQXoktYeBTg`i9lE?nIEU9t$o_EGW`%zz(>Eq>V|w)YKii|oJ*T=1*sZdg zTnn=J>f#?;&PnpEcviS8{M*Si-Yx&Sf;s1(vHHQ8KQ;WInDA-SJ?|7gT3a%$KWcJa z=DOsyqs?brQg)v{V)r`58iKOZZ?`Z#`(X6*C5hDQp& zGny=vw6FbpcHcz)*AmL1Svkziv7Z0TugNIOhxSbDf1T0n|G!##jpB^0InAY~Gn&tT z*vV^{>zu<~67)cn=o>F75x|-Q;eQXTTX3fbg zE(t>2MVwnM>?%^XOh0l{tV%6=%PI*2Uv5KD0cnTK%v=*+ugTlOu59vL_IBIsyBlsM zbag%Yy?g#&);h-)xm!&#TkAf_)K58XJ=?^0@)q~=^JdiNHo>i>U!{hp!e zN26ok>`LvU?xu5%OOBqsVzfOuGH>O!G^w4zJEyE#&7&84`sO`_!wWk;t0z3=jXwRs zL1*WY;BPAP`6o`ee&+U_Bnu(WnKK)XUXIcW=+$^~F1sx)KE>ksZSR6`-@TW4d=ID1 z;;|FB$Rl?)ebODT6`l{B&QGdx=6$hhqd>>A(%M5;cs6A&neZr*QF~$Q!i(3f*E$@k zljNG0!0{%p_4;b5Wrp+3_cS$n&bu1E$B|Qx^=!e;i(O`{y%|FBvOM#0pSl*SmYjSc zW@XiVujg6s6b8Pk;9av$ds?Ve2-j3dTGdR{S~J)QLJ9Yi+0}uk4OzY~SiA0^ zOI7p{l^2;;FR}QmZklyf=dJOa53Z|p`M+NKmt?WWSj{=D@wwV(w(RDrcPz z@kpEPjNvz3^MGaEo|Qd<9#>vmyHXl0dadum!D0 zyF}FM1zsVck;cbG?`WN>^_ZG6`{$-ycjAhoR7xm6*y8Xi?q0Y-zJ8sXUDH{?C zvLhy(>TK<+s_ob7&?@1KpZL@CjrWf>1)06GJR@CWgRbXwscAm6_Mb2-*nI2H^&fTj zxi>HO-BQzVuY~i0t5)Nk<94&|Zs}Uw_^omIuYXRblQlL=TYPBWpU?aD-#^omU*-iD z{tMWC_gungbXL!bUFv|mzx(?)L;G!cJJO-cib!e zvOej-J-G>MRZpbt`V@B{^H#oS?RJ*R`D^l*nTthqX>C!xd|0_1D zJ@1J4%~!iYR%efQv2gAE1%U^=PCj?E(peDObnyH4N9RKM+PioBJ`h;8+U}hDfAQyL zc6tv(tUlh{E2S@JYlebzS>czS(i)Y%V>!p8xzCPcc!RDBAlWbbM*0l4JI#qPq zBmabnH8ZbVq;&GqpFlbH$sGZ_=hsx&Rs6ELQOr44;oP!)lNPId*WYv}*Qr*0o=Rqr zjpCHud(B%;<(WMadaIqmB(`u{at7Zz#~T|S3V1nsWU&3dxckI>mzOvAxu-5kUU!LO zQpi4wIebg2M0vk*EVbO-=gR&$BjQY)lFpxpyluaY^jOy2j%a(lplKD`TA`@B_x67Z zzoR5IoAXzTsB&BW5{uJMj!o7$&uXO;Yrdmo@7>qd8IPy+&fei0Vaj^N5 z|7dTqXkph#^wmCMzjWo(l<3A*DVF=!Zbm1qp8kXHw?wh~F~z@m&l`087Ao?tzxk(W zT6vG?d*N$+JUNa2vl$O)KDT*z!sgofPT6g*cNes$1Kg|MGT4^xOY)s*8<{XS!s3`h9k9 z@wxJSe}C8CpK1Q@$Dz{iG=6hYyP3JJ@0icQDc@Y4n;GA|m93`yGDB&emE~2* zPFI$tQ!iGS#A;e>)VebB#Y+EKu@^&X6~A6$&1QW!wNr1Clzi8*v(tRery1WAyt!tE z_gwzCgIg}1uYM?A5x#Y6?i!Qr+-l*i>pI?aTsf}vaqZW)FH=nRO%>vqa!|vW;W5j) zS=YF0B>ewI=?A3q_-y&678ujZe_ZqTtf*@(VUfFo*W7))Gy0`-z=P<*!V`9ucW=!; zxHn7QPvFOFeZEqY$l06MoV|G`=S!mA)|{-D8@4@*j8+c}@w%O?SwClO%^jcHWm$Ju zZ*T|xxdPae_vu%E@yputuA<$jmDL3+(Bx>Q70YHG%j znWxU4+a;dP*>mcgi;m!SJJ!ubo0pY-7EM#Poqztd%o>gdIct{ayKi4{sXMeMsQC3x z@%Avo&*^=MwW>4M9jvTd<2HGL%9VAG;*Vr7iz|pIgc=90PoKS2%VK53B5~`xUEEWJ z^^%?~zvL;pTdA(JxI0n*^vC@PM-AE5XB;^eDWk6nHg=uy zKI{FnV5ZfI$Ti+>Rhl6MMS;z_ekXlwWZxRTzP)Jf-M)v?8#Nu4xhxX?v@%SkY*uX8 zl=Gfy%DeZx5b@FQVbGs(dFg49HeK~op}889l-A5$`+{!)ci_P*7fU`SNn7T6s#Ggw z_O>Of?QE1g!rY9Ru?o-$wkn&D!+ zfmyLtFkI;IoXv4>f>sLe?V48SeY2WdsJo6k!F=6vh1T6{d0RwwYU-yr_xG_Pjp*UB?k4wc;Zk>QUR* zbj`F)vpKu|an7y@-&mC0UM>7?RGY@(+7lBk@kV>{#VsqZ+H5h5L>sXiW= z(_bZRRzInqf9`eH66LA9$N7p~e*`UEYcut72PeC8Lfnl-ykC;{2PHKe=c=eQJfR`* zF=;nfPtmSR9jeUw51$(C=?ITdKBZAMnI*XV5r^wdhoH1wXRq9uE^oXm`b|`|=+PkA zl-_!F<{M4UdVKA30&G^^i!fRHs`6dSPrE?n!bSDm$C>a2<9@@x&nwLjG#-p~Gl z{ZG5`_pRz7*MI6?YxkURW`5%wtzY?n`sWM&h)<7`(480er=Gz*WW_0+L(^RDta)PE zdP4eCWMFgelpn4~j>=7W&mr_ea@Sd(^hqB>t*w1y!{R=Q^fmU?eeX0^So$b=VkVEd zO2XPHg2#)qFKh34_G41os_B9Xv-}O(Z)~5b$tUb>p(QT1S$(dDdy>gQ&#MNP=Nyob zF|Ba6Kbq-g%d#m4UD7_bPHPb(D==QZQxJ zlpgLMPeKmf0Sr3hVc4`Fh3# zU4JlH!1zkiT^Y~vvh`kS!dF(hHZ}@Q2-`V9a#{KVMYmHM%+z;hAOBan(BR_M=u5k+ zLSrrFE>ylOZ7hCqMtADkPgks#FIpqJzGdZ`V{Fc5D<(zjPrtNniKWTa6HlvFP3$dM z`1Nw5qTuQ0v#;)SZ7YdDYJ7%TLhTj-@ir)oE9IN*9e9hYSW^2%Ex!XLpO{_*A7k1y$xN$Y* zMD3^CqtELmW&eJAx^??xE%k6=f2NRYdScT~T(}TtD6(PB{oOp35psc1Hm4r$NVu%M z$E(6Bxl`ovXN%{4l|B^{3sjB1T>t*XN7sK-t=Q~cdq0&p%nN*?dU1#1{q2_?T8chW z=cyE6zk2=OM8nu42^W{Yd)LBsYQfWm;yW+Q_9%LoT6&-Ngm%^g*_lgx!|v^0CGwY1 zuvs!YGU-J2w5j%wrtM;9X)if<^2NeEe+-Y`nRDgb!V8su+aCQBzPh?=QiRqX$qCn= z>OL)M*LePY|JtGm%SUr3J`?^Zw`oJ~=aRn{MKb+#SiSq69XU}Xs$+Mmxl&wP&3*UR z%bPDAiSb_7;~J!Swe3_UG$ zIo0Ra=Ia`n12uLxH?Rd5e+jbxQsl+?ez(H)pt^Z3kq0V%7&QJDHWZy~z%9DsP+>$z zrUmomjZIRzic=F`HXgN@>TmCN_~o{~?2~8DKEE<++AT(%=e`=<#q*E;jNHPrU@eQk z^(QaWet(nwkpanPLuSvf%64zRH@~af)L%7uuI^)@ik*iHUqr@Oz5JwO#=l7WO(kda zp>?yqZMZ$@+P!}5O)qq(eBU_D{HWFY6RVGu>7MUb37RIdbHcn2h3_{^nsjnxGmmp z^&*VDRzK1IhUgcKe{U20H*B$uiMlQKjcqUUj_y5t5Auv85A0nxhr{RMy_){G3VG>| zjCse}cks(+@^6jYW-VvclYdjt(q|Um+x>@sm?bY(`fm2_`K;MjzCW_A`8*+BU+zJ) z=FWA8nkzaCKR-^iNjh$DNcL#5S#PaR0Y6jCP4$TISmO@$!&goJR#Y*Z-&rJ7GRbd^ z`OG=_AOGeqdmL?faL(HmR?_EhAFN@EjE?Cr@w)gas;knm^^F7PHQV3`GV7nKB=0=D zUS|ERWz!$C|5)gE?D_{~Yu{s^*G#Y8&TCZi_ebI8Ee4BtK0WhXpZ;-K^~JhBHJ$03 zmOrW6!uRN|f%)biTf1g<8*Y9y>8ShPA3HWqseY&PuWFNecfG`sg;Z}ZOu4w(+#Tw!I#~3y9S5{lSsB3-2x`Ekx z$rZE0zP=ZAUv~?oJz_nRsUJ8$@^Eog+xnPeA{y4+Y0lg z)tvr%Q*K_l>w5ROMWb!!{i!{*+unV7x99IB1#KV2mEtDt{0nnp-#YfZKHPF^&74i* zbz0Zk^>49jrEJ``&1qW7Q=aLYsw!J`a!Xe8pZB@WY%zaM#D(5>Gkaw7KIXiNveQU! z|NV5%0(I@xAr9fGsauumerzr;+HT^Xz5e~?y1BA)ztV3Qt1s@EUmbXOR(3*VYsAd# z$ivbRd*`mXEf@CY>zb?VKmW75oU&Z(^^CX0He1hs3t?V&=GxRuiu{YD!p26`!B6f^H*Gb=-sK65&M--^dz^RfxU9L7CVy$m+$FkOGrUdTeo0ulZ0!oQ%epC63KxaGJj`=l z_se_nb*1mOp4@Z#l-p&q>g3J0Qng|`{U@_G+xglBRyV4b6zW}G^+D*;i|rkM&GPFN z-^U%TzOzB{Z0(1wE7IRyzU8^}kN>T_T$OiB@k$J_&h78gtm3>U-12>}t1olU?1Bl3k*!^(RU#zQAhD z3mKj)=XHW9PuW%lZ#x&1>e2a8Lf`yp>5aT~`A@yft>21To|KW1J3nJrm{EPI#!SKO z?E70&{)Zm@7b5opA z+|C@WoOR~(=R>!v3+Lpm{QcBg%t-9JOUJy7WqgWBDaLQ}4rx(|)NF zcJrpsui~PL^92?5Pg;Js?Vj{1=5$8X{8dY8Ybw3=+MJ&L{Hpgf{oj@Eay!$*6QL_Ys1){u-cxtd$;0C zi>+O7(ANbv%N{&cWwht#`1Q1X8uO2@3xeKGc-rQ;)67O=eS7<~`dj+>-%p?4l0D_t zF{{%o?mq>hFZovf2s!%o=dCp_L$=TQpKXz75_~i9=yl7k^;73Zcb$IP_efje+TOlR z_e5XBEqhkCTB3;i;HzcD#Yx6jcjnu@`Vj4z#p^dM^x*}^DOV#L-iA(ns+VG+-kx}Y z^IP^O?%9dWYZqMSDore(KehPIYnlB~b&*egd{*wCAMwUQ`P*uxs-{K-3i{->X}IeCpHpYK;U`NIhs1k&GH z|8}Dw1H%q=e1kwZPooIVFG9NhOb4SlD+*2nlJsM`%b4;*l@BC194(r!{n_mQr^eiqB zX-S>8WbwVECAY8dNhk;jeX027FMIBHrmvG1-HG+7JIUX~F*#6b>T%0#b8g+j)NR)v z9CozP5og`|Gy9VL)^hj4pH|+%%ZynQAM@Mo{+n1P@#JG&5XF8~<=x?*$fqfoYYO+By>&PM%0}Niba3Slr2)%vt)k z_m}Slh6A?3)!Uj&{yqJ|!(#ee)$`FVquB?i^-lYyxUr3Gx6uJ}2fgZ$?=d+ZMYA)) zKB?+0jGQTWC2N9h;M?0dmGM5A*H&{5^KQBAIcrL8tNF}g?+dxE z&#L<@uF3x3DKD%u=z3Rb6BSu{QRIr5r(eI#uKT%@_N!&YF8tfK^|(B1u8l&qk=@C{ zmVGB3!rXuX(w0L3&qD_{+{@+w^bTdv-XV z;;@=KACze3Dj&DL$H>6YfOq5&b!7|2s4{$^m?0$DObyKqmv$AYYjbGR(QKK1$%}jI zt}TjzNn1j5HbiN7y!jMl7LZ(fTg5qI)|+Lsd4u}xPx5bF&bHp!fo*t0oO zQ_;I`-kiC2XYTyI^L*{Ib7$w*|No=iV6rbUpk(($k)xX;1$iFtimg$(^L17M&-E>9 zf934W2;phlkYyh8&>-eoSDES4j!BXJbIs08xVtBF=DXZB?T>kF#l~3D3rzm7@jWB#w z+V?q#Yv!U;24dI!B2!gXa_?A?AhSN8?cKV_^d0wHJB6Qk>=it#b|Ck#;ZmV}GPQ1N z!zvFI=&qalZ1&8TFNghOO5{El);H7)0T;_Pip^torkKA-WSU9;aC((Pi zdcRq~tnUWq4doy0L>*ltQp>MvAITAsJfpp%i(kV*z4KSv;Zqk|9y_;X9JD)eGji37 zO+Mu|^K2zg#xD;j->~3;vW`*As~}q$Z=T6_U!4~I|CnRi#zPktzYMHtvTR-UJ>^3? zqw3SUt3z4MHykVYsPoj)%%fHI_DZRB<+r2Tc2&s4gwDPoddI!5@$L5OQ}Uf|tzH+H zw&Q{F$&E)}-akEk`joAGZ+mPS{XYgAO#d-Qqxyp1RiE6hRd2h5dmnqR+W%Pdu1{XX z(mU%mNL*>yeP`pLg3XuT*#B@%T%b|y)R}0(F!l1!t8aF7EV{F4@vc|pZ>Jetx!tsb z)niY)`=y;uy3YBfZ!@lKYT2xDRd~v}Pn&LDP_=D6`EkaL(vH)PeaUeqmzO%hWf-w$39LI*L!I6pw_cQY`4GdtFHdN4rYo~{A%-(D@-T; zkqw_6Fu&}e^p}sn1P+?4yZCRzn@YQT2{P#S?+bzqUANI68|M#Tgwb?gb9ZPvSuYR<7>!~*R z-kdVe1^WAz^Iv28Y}%{Bwr|mKhl9)OPfxd+<00o*!YyYv&D6)}O~t`e2Q8lcKYH`< zoc>~?&R3WFRGjWbw{5cFm@U_Ot!>5WYe_Hb!P{L*+J0!TvM?~L;6SdD(5LjZ@XuL+ zE2g59ROs2brQMEPhYUnqo=4xl@#@f2?(4DAHq8npl8$2F0`6e zyvg{Fq}`rMoBVQy>@ys#pQa~N+>TJwnPTq#Zj~|nUDJCAty>#Sx6uZVx@tSpNEAGK7jULO_pmT%Lo?KxJBO3EoSk^-1s z6<@Lb`RDOJu05TcpL8JWZdbSUCC+7FV7SkK>}mAL7%3D_J4063X6ENXOVG&b3XyP8 z`|@vwn`d)IPCCoP*}@uhBgd@8X35e_O%;|Kn^a6@yD!{UxY^p1&CK|;okGA?#$eag zSF>u5xF6LD(hglzOGbH&-1KWm+c|dwEJm)YYw9S7xZlDJraxYCO7d z??mQ_QK9Q|&wcr>?WuZgp-Yn2rp^Y}^Gi4@HZ^?nv6|@Fe%VvisAcveV=m)q2^4{UnGu2iUgT1)iu7dT*X?s)}*LB zM1IxL9alo$rtrB$s5l#1$Zn9BcH%)(>z6e$pG{8QySH8ZppuUB_L|VUMpF(HsfRR7 zZz!KMx%JMxfJuU0b5`{nQ(ifRL;K=OuU93ijZgPoc*Es8WtY&p!b4gSz6*{>%)J%s zX*;cRnfi^I89wnDr*wnQzi-i6aqP$C=BzG0as4}LR$^9J{&}xIqeXC}gXFPi~cgA|Qp1_;)JC9r}$b5HT!xhD5p8Te-CYjfzJb5mK3l)vGob@!V?QmFFxl?=Pm(_`Gl|xj`b5|IVw1yc@m;E?Nsp zsa3em{K)mdtYWipY5wswJZ?eFIqxj1)X%4fY4`n&R&d(-YeSUw<%x{mZ_Zh375ueW zv5fuY7GCX64nHp0_QidZ!zLT7dGq|IOTzOxn|0#tB!4ANIOjRl!I5vufvkHIKFegC zySy-RkHn(52SIP8P93?NRjjsey{o)bSB4>z?jDn;b9daU5uRSIB|Kq?8k0)<*xgj4=YZ5Xs~B$ z)hDxG9CPaA)!n>2%QZSJ&i$Nvrz=tGEoX7#y&2_OeH14ovh)R&-)j^$T(@jSuH?J4 zOXZ8U@OvMhBA0xrTsp$naOro0m(_(AgKjcq+}knzZiT&GxSQatN55zMXzn<^jKR7+ zgsFDv-L`zUipHJu8!tESl(%zQY2Q`3sNW7~& ze2hVUKL5ePA^*FVDc0zn{J`<}k5FG*;+-0exoMA#IeHgOd~$6DPyeQUk7S%Kx=gXn zO4)HBLOsl$&lA0=On7HQXI*34hP7#%l+=EU9t!F0+OYJ)*=dryPDCE?60Pf2*&&}G zX%vabKkyN-vYE_?LttVBkanB$@s_Q@@7)k0w#Sv6848`3Ot;jQF`d?hNjy57+as)af1JNwJ&6_Mm)^_!2d)154*AM1&_klDi=6< zlz+7yPqH#Qw()??sb?E6*ytJbyFL>%7e2P}#*Te_1xXFk_1+&>o!-E_;Y56`+^Ij_ z=MM+{ca3-`-){Ot-XdAz+k^G8q71Stj>hW;H%ditC(H@VN}3ax+_ZYew%%o&tdbcD z*-ZbqFWlJnn@i-&6&vnW^@G91jO(PE7ZvTfSR&){+H*sc-`T_>-)m2#b};e^K7Qk7 zqW9pZ+m4eaJH;=G?BDn2*i!z-8%1u&9}WI{N(H7<}8>p!^Yz~v1)&h~re9QuE(TVUBw>Dk8pI$do!BD@l=%O9To zwx&EzdG?Azv2O`>?;h;DF}t@?%IZ|N%`{hkw{;&H*69??>de$*@d&kyo=R3B`KT>=8wZgYJ>w9>Avt9Vx z8=HH|CQnRI-gW=U%~odL_9&<8S;rL5rhhW_ndW)wm)gwF4f9ejcyoL4AK;rez5U1K zsxp24!*lPj?5kb(HE|ouy;PL~mc9=$(r3*Z%Rg7%o2q}_(C&Qor>8sH&wt)mQ~B=c z=Ng6hdB1JSC3;2sQ<8{=$X+9~Sr*E%g7m zVET>=r)^#se+cFK5*f#~?tr<)h5U~T&L7-vdEx!X1^1up);@puvu64}`A?N`C#F9? zUNcYp^V6T2^PexRsjNHqywWm$Ui_!ZxRcYLU;iL2cYgKf=NqML#QaR>HiuWLezN{y zd;UrGXY)*RpBomTm3(oJXZ}(C)0zJ1>(AL!c!ZVGtL8ReuT+!0WB0OBZc77qZ6o{C zofpI_FF5IaQI#uU6#r$!A9bX4pONsXUBR>7?0@iZ&D0fTog9mmm6d0^C_Yc!9=su< z^X!eMmYuy31?STng`_?7%4d2P=xC_;Ee}u-)~@~dNTv0N*Y_g6g(oy`++22ERYylp zrO8=CkmdT9-bLOTu`wPtew#XO^wchNj(8-g-)rse1B! z#j4JaimRuvTnOuRj(9Dod4o%!l6_zG}O-bL){?%5Ted=Pa$>)9BV-Fx$Pr zXHVwZ<4?Z!9D18~Nw;&at-G6>xv4<3%ES)&Nk=*g@46r9N%OelwB=%|X757@ExChI z*V;vUBJ;ScktXjp?~1pDK7S&{S?f_z~C&4uW4$9(liZ1x}5^Cyf!m0vm`Sy zC$lOwJ~^ipbaMq_cUZQ+u%n1=^1|-dOu8Ie9Q(rFIIsNLDHgbLK}MwqXJ?(^X`@54 z-tydhC-uerC!;XyqRwAukI1L)*}OYz>C}u_=iZ&UWBGkf@%uO1-^<%EYAianAUyTK z&6-<hMM|$n;Y1xJhRY6Gd9_Wf7-fSiL?IZCz+?gIbplEZhW{eN#bP4w@dn|v#0+Hi2Q2nz9!SclS#VY z{q@o}!OVv()=sugmta#Wj8mvtevEr^(d6sm%o|;VdDA6(M0f5~QD0iJQ&r-ZvA^`? zKFrS#7rNn_kH_ zWx@Naq?GT>cip|{M(C_<-Zv$grx*Ah4~<#Q#-CYf+AeWr$$9Bt@7eB7i4R`2BCYf4 z(o(-wXT@wUY&lf0)$)z}IZ27*Ew7f|=6NZ$LTctV#n*QZq=RV=%F++WwBBQ(5 zsyo~hLpEnu%+U%m{&s2Z+`~*_2ZL;1{k!zsoB5;DB)8tQVhacR1cm>i?RHM}o0`v^ z-}$F{NwKxXoZCkaMBlP);FG#484%8Q@rL}|LkfM&4`(~2uRLDD?|0-zu*l>E=~mzQ zUfzuP$L*JU_>OB{w)}|_o|}Bkm2;N=-Lfmn@JR&EO;N{76Yjfx^!2t;u6f_}QNr|1 zYFChkvq^CE+a>vLe0SWe*x{m_dC>P)|KqSZztY}>7)`%_bnagjmHM5NR5;5Ir%Ya3 z`0lU%Igw?_bKC@tqkWZx9tJP1l5nzFwBIlL?ia=tOSY=WJ81?621y161`!4p22|kd8sezy>F1{J>*(j{<{BKL=j-TRedHsv-TQf5-cyTZ==pjsHU|=xdLpB&KP*PKZQ!$3g z(H(s_cZ=*rCI*JdtmuwDE{JS4ZbutIh5^tGp0P(~uQCe*13M=JgE5MuKZ+n5jO1wV z)XLz3#N<@Ou|4==lXKw?z6KTshW|Y1Nu^X0hXp>F#Yh2y&u~j8#y?d|3=AGD3=Fy` zt`CyKVK}Diu?Nh=i;2tKm>C!t*cli=83tjxx+1dC&hT*-73>BjFaNarCMyGjGaq^Y zY*Rrt2q}a?sf@VLEjT(aY9|u|10yQ~186ZW!j*5dkPXKeV0Up1cE;gghn1V6cQ7$9 z{9|EYP(>KXz`*cR7q6L)IF0mQn5=)0iGkrND+2>)x(Hz^iy^X+NMV4Q2(Sk&+mj{x z<(U~6{;)AHXdujFU|^6j!C@>Bd5te?{xu6$1_oDt^n5(goETH_CD=z9r{NcKgDx*b zI9k;fm)ZG6C1_;?E*n6@pj=2d21_7}XJBCPa6z^KR#IRO*%?UJFUp~qBItq56kIL> zRTjt>n4$Wa%OBY+j5G=_&(W`ML@&gUPt6BigobdiNC>hG$PG{I{zg8}8#SRaCLx=I zlu%JKD|Ta%4@eV5xQ>B=VPh(?v0w*cHv;*bC)8ZIy#(0^B$pwI5bQ=GALjr%APV6y z;|gRW(IW=CX~;X;QQHLb8<0%{`;pLr5a=L(tg#9Y&cfm<4T;M?1H=p2DA|n z5eEL<$OfZ_0isoau`LkY?Z{gxK$~e0R^&~8Tt;TCG*wu)c zA7pbuMHO;y7BroLFjssHvbl&#AG^nqTi~F+FT$Kn+mOw{o)>WUgaf=;*+6C~Gbl5h M<7QxR+XdnQ0DuBM?*IS* diff --git a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar b/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar new file mode 100644 index 0000000000000000000000000000000000000000..a269f15f7aaab12b04c9a2ba8fc4c6572b0f017f GIT binary patch literal 47464 zcmWIWW@h1H00GzdBpVP7!<-BZ48E=*j=G+HZu&4~91Je;Nj9ZvXRcH*FfhzyU|Qel)8uT+$&+Q;G?E5Bk9h=Wu&9nV{!u_w^w-r|`ciC1T5PR!eIR{5R%%uzr%xE5BVPIfL&d-JW z1ycnmAV4aLQd2UE;p#ZRCUI?BTPMZLz_5Ugfk7K;5=31PNVRiLW@=uEib`T$eqLH> zo=S0QUW$s5UUE)iaq-lU?BL4|BDSX&Hwr9$%FCt2_QH9k>&H+RR>###ZMe?-Fx;Zu z;%V+POY7I{KMj3cuX=u+m8_rg`0f^mT;|1>U;37Pf4B4doa+3#zyCh*IxuDPWL?v< zw6x2QnxnYo(C4(mYeqW@j(1xgHj-f46fImcnG zlZ7+6qS78uX~`C=s5y4;=}*%|s}fpvtVs=8&GYxA@SI6f#{}|@YAA)wjkQ(0`9o#7 zZ1%*ckb71S=CO!|A zb$Q-NdhvNu0hd~4{D#=LtM271?)Fa$(Kssb^Qx|%M9KV1*WMcVT-*Kkuq|)!jA^<> z^VQCY}2cg}1jyi!R9w%Z{2>mX)Lxp%PST8o0ryv-;+YGMm#Y59)sIeUs06Gok6%`pM6V zIJF~Kxyw#SF6E!ZzZe&`X1rn7t8MxxcX4m9g0tv} zw2Y32Tyxu>n1m|44^wDP^)T9a-oxa8z316#{xAW_ukTAgO|od-GAaG+kq_UZ-*5O; zG)ZUU!h)V#ui`=rW+o{}Ss!S=?7`Ak!YR<%9=AcVevbS9ZsYc!Tz;mVSK?$l-QH_D z+kRD;cQJnH-ZC8pVV^&Yu*?d{VwQH+lUA@WFf8O`V9-a)Vg)&sh|HB+np2YLRGO0= zlnN3zgl4nQ=y2&!k$-%gl14tN4vovA_AE8!yda?)${Dt>Rf_lFB$gvpvs)P3{L)^e zG;Iuh{r3}d&5FN_`+fvOvaZsZYPWM*{gL{IMe}ExaL&B2R6G0J8Oz_lYOCkZ-T&|B z*K&r!n&S#-J)0r}mWBEzzK9J7yK-qxW`x_p3+JYBZBjF?*b*>DamJ?YhMU(!X9aei zeP%wTY3qVDY7#Yyi*q#9oXb}wh$O3h7m5xr+sePQAm(c5Yo@~<-J&)Y8#@HJI9kZRL^ft-nB-&g6(5~<797DM#po!?ea+$a(IX3%foL51D*eSkKi-T2H#NJb}JUp{tmF07nBeh2gJ{8ES^>zhCX}pfS#j)Vd(M|Kr zwIY>QMzd_2dVNWYafREF%oa{hi<1j&TUU5o;Wgyh@pt``4lzdMC3C&iH@{2^+{vk>I++$Uy7am4z z>^}P`brzo;8?IGF9dJFgD#I)AkjS~8g`AhS9?;zC zlCtO6g~r|C^JJl+r_oxQV3ty3=D5c$YZ%5Gk zR_Uv=bM7D9tu(1(y_?Ui3!+MH>nm6AE*%v2VzVf=} z80QZ)f#o-i96tZ_uIbIcDE8vvW~l|{zWKlEg&kxTG>6&0HOf%7{P5$#{1@TugAGvE9wfDMeVZ%G zw{@R?A^7&u{Nv9Xf9yvs^ztijae&%U7Z?~ARM85(l+0piyCOKhsKmc0CAA1r)a9Ob z1U0T$FW6t(wykU-_w~FOX&Yz9g3t}>9WNDh4I7tKS%}LWaPC~mTKJIXb9th;Y5+Nl`+Ns_q;;C{ea_+P%%{Yo_kyFfaJ{dCSVR zueGml@D>yfs}Jm&GND#A@#4wZB2lw8uke#yt?;R3LPFn0rGOKjFJ2V;AATZngeP*+ zUbF(eEWW@^oSA`PCp!a!EQ;TPQ%n2{Qi~ExGV}9_A%0pL8XbH&K;YipCtEzT40#>* z6bi0b#1rPm?(QfeAS2@@p%r&w`UN=ypV?PbKCCf5_v6+3j|b%+^cUSxSvrNOYkIrc z-+i0YzgyqGnYZu%k6*kDY706Nru@CTKuP@WESDF0r!$r+GOHKW>hF;Zn=`9<0yp#V z)z?yMgV{Ko7oWC1H!paV<-M|%GOX*i+m_Z^a_X)%dDI)sF?m1lEITQK!_r?j?S5-{ zG4DXu7K62tN_(tb_lbo5iSi9gTK44H>)jJKM^09?yY}V%>hmgGJsV!-<+e_g_m;3< zmS-FoxzE1po%sG9`)#jIZquopcxRg9A}^Iy0(Sqa8~;BE*1Pldk)CJosU;fAVyDS& zdBO5=Z(H7u`;OQ9tPfv&eYe9uwy;U-&|Uki<%joO{c9_>sbbZvfJ{nA60A6dHpM%gBX<)_3KOnn^jFJI|P*q^I7+u6=>lxR_Oka!q)WvP)_U0q5auQ@bwS+8~2&NM4?(!HtfhZo$D3`wr$J+oM9_S(yl z%| zHqBst=f^a^ye!`-5_b+QxtKf45m88dhOg_j>CG~ZYUce~D= z7m?z_zDaSeM_9Va#v|49lYUEd*G618Qe>QI^upxh8p(14R@05MT9#~%63+H?x&7Nk z`25VM+E{n9gdiu!3Z`;>+t~dYWVYOHB)Y_we4dRQJivMLnEiFpV%&_~<$iVQ6 znSntZB}X}{fcqMHV6rtR)?e6B#CG%A*o^H?nj#1F4~Rx@-{4rg`r|=06}8oiOigY4 zwo6CMc=w=g!jH{0jT)Wy2jo-tY`#6^QrO*;w0HM*KA%(V|LppEd3(k;2h%jlQiOKh z+ox~S_R4+zdw2GQJxjHZ<~=DoD9N=<{>R3X8zRbMEH^lZrfiSy)pOpb`B<0Snr=8=k@^%*oUP!N~n|#mm;qyCUHWr(#3tzCi z>#FCTxaZGg{w;OG^s)_Ay_LcoS&MbqndQEeN?iHnef5`i+tgnsv5T`Gn2B|1Bu`A# zn-)1Qap%1m>&!N^tXdVjtWaI*V&LM=pEqazIJLo6@9VzuM>`8w8_qQ;tQL4@0E>{@4E=^4v6Rb zyycUE+^@|UhFkYtT(;1mTG7{J1&_fd?-w%`7t6U$X*-UfAXGlu7%3 zVff+ao?6yRZ=E>R9-_6Y{rAgSQT`7T1iI>yJ?ecTO0Qk}6n?{M%Pi~Z6*Y}-Z*qmL z`k>_8Ba)@uDm5=)$QNLPivg$sE`z4Z>y4g&t`Eg!^3-))g%469~o=p z>VGiUO?muGfXg`KqxI`$*Ia!w9dvj#S&SpS84*O6OE z+PW274?pemyId`sW^rL1d+!Na;DgpK9Y9&5H|bu)`+#NZ*QGj@GN19^(Nt6J3h@j zuF3wGFV=uDf@{N;GmTd+ohwnFTwCC9x+d*?J7b!r z5w~MwsW{v2UB|05AKP-Y@A@I|y8U6tvHrAAH}rlRygpw3aoz*@SHcH5x@W8H+nBQ? z|Nj1();ruqWxpOwNL+N`ZehTJgL{nF(DLI*RPdH=7#s>h@I|O}>NX5>BgSTpWYGeK0sKv+?B$?`O(xOC-{M7hVZ} zcd^s!phjQMO_Ntee6`1Udct$GcB`a4zHwjY!pY#jA#*mYe6?unk)x}(o6WUzzIKhl zqQ|>%TIBhM=iW@+FwH1@iN*P8+%tNYr~8Cy#awiqt+#W^?wdJnr}~%dSTuK6v;NUg zwsnc$W_ejmo#b2mpmcUxjM0ys9)-=dX@=ERa=t~IEZg!Dx5?WiuPU0&{QkrJov$Yu z%dVdGNqDQ7@=+V>dxzgXeHz%?EG|*qa5K$Yd+xNPi5pelyt6-d=z&pl>7BTK)j3lh zO>EnBI=D-HmQQ}P=De)c_YMcyTu*&HH+I4F^`AV<_Nu*#_B)=W#}hIIy?1)%otfKt zW(EdV4h9Allx*aepOT8mN+{)U$ouHqAtH79TYRNA^~yJ|4VJjPE_-2gY+(D{W!aI9 zS($0U4i0lOt6y$ft}k@a^{yGr;Uqjx164#Aorj44bTY%4HQ|n^}b2YB^ z?%i@yde+o`>)#%J!seXNc}-&76qC8rZ&#b$JF5rJ^r=C4ixu?GPkS1#|WyKMc`cJa}wa|Uxf^1Npp z+A8St$f`ME>lJ3b;KIwyFXA{fug87b^;S(`r>cMW?ccid1V7jvzVP|?l^HIZuE!nX z$?KYNY)$gii60ux|4vBUVlB_OUGmtu@a|n_XWtPiX#Y{BCG0jS@tAVLw$kX^k6-Tj zILmxbh2`atdLaY<*f(1@YUS?r5@sy-49?w~wr8D`$dc7N{-;GvnQ+Wq>0RR_VZJM2 zx7S`zSv2EKZfgr`jgC^Ezl2&@)kPK?go zsVh0_q(bU6_Xla_gmqnYIv42N+OS4EcX#3*rK#;xuYP?MT)A21nAKU0;M+S^HA5?UVMro8QmPf3C;6ht(>%sjK{2PuX=32jzlp zuaggBl$0dg7@jz6uB@8z`H$OS0Vk8TJFSUE%qsc9Oo!HKn=XAkLpeJ=Kj|II^keH~ zcFIrQuYBsmX~`)b`NvI~{;=-of#%Zp_O&BjmKL`aPQR`+_erFULgVBk8(lt!R;_=m7-F+*m+r2x;5@V4vfs{K4;JaD zXE|MaYLnFp9m%4Y54zrh@xKmDJ@;Uq;h$Tq=Kg(}@jKT4-Vn^Tv3ug<-1r??@;iDR z3yhysTo%vU?7t~Gz2N>gwd!7(^}i3*UT?qq+3}6r$-*<@@7nDOwmKFF?sAp4;7{my zw`=kIdIr>bRn*w;Q~(nL!yje_20@epAqdh6>kYo0ciBN;u4J|B`pYJMlXJa~x?Y%~ z7I0PMh_5Cqmxw6STA_o?`FREtwU4h85?`6G@Rj)o%f4oIyM}(J9`l8rO%5kMrJkE- zX`G(^_wn=X?-@2QdUN<$TfQrAnrpx%aBA&?T8kU+)iW=7F8b}ZaUBaQK|C%TNy$0{(8Po6iInG|(aa7qa?Fe`E-p5z$ zJDB=i&6eMIob`5#{IR>ortK3*v3_%`uIkS7WxDkjc-Ze8_!hq^b@l)CZ}_)OR^j?1 zad5@e`|S&H>ALJ5x958ZiFQJ;0=QPQ$S; z*;3G2=6RyqXD;C%I>(Q8-gqkgRos!2VgIZjwB>*+~Ny;s=8XI z{BLYKDm3R%<(5^6GRs1n%il1i$4)8nocOcNFghmfn5fO{sfSXpuLxQ16%`m0<$rqV z)$Gk7D?L|fZ~1o9>4W2?ucn*YEcqU9i%3`GZ*O;b*z1$p++q82*(nBD-LcR19C(<&`h)8AEU&7gcMJY-qb5It?0r8v1sE9QEEpK1QIj8HLIRQy z*G6+z#9S@4dw%xYvzMMdOD8_5m6i64OQ}2@;kMxDvP0c1sk0-F&6xPiQ!ZPqQcPKs z&!f*n#v{T>VS)fxi;jSTTE)adC!tF^4%759XPf0-UgkS}iSP6+TV6ULNz?3o&Yr8-Qh!3JFLIJ{lB!fSOQ+P^!xtq?Z}z)9E?GTAMzKC+!^B_g%FTD` zvkioNwoGW*b?4c+Uw^Ocwv|#)UUy5fCt~;Hhj!ZAuHQeN7W&=&aIc}+TEoK*;vusV zoOVjDI*{!odE@6wwu8l-`&PeSan*m$=97DN{ramGav+BJ*#VoNPeK{#)7RTS=|3L3 zq2bGyjq~g_ZO%(q^ENJzXX)mR;P77YH7NRb$@^3l>&?N{tIiiO*KANa-S_aqi<13T zEct2n7iYY>o#0@*W{zV1`I7xtBR_px^2>Hxu@aCQ9nK5GN-w&{-ScRX%Rvk>Z0zVfV)F}-k# z+l9J%ofDBWyj-K}EgCjBO+55<)sppR12lvd?aAwJJN(b%0PAi|*$@pS_uafNnq~`r zaS6F(=_tbF^P+L@nce;oS_=ekiSFR)TEYM1Z*taE<4Nh^MpHPpsX9AnzU5a~qaD>{ z@OA;?-U99V3Fo#cW>0auQ&_K&8K`s0Xop~)X3&9acaDi|&3rfSLBf&w9>+8meA{u< zE_&S=rupaPJKfkO3Cb7$yteZ|l5|D_55v(VI>{?`D|mc6qFlL2dFiunDNE)XZ}+T7 z=5R_a5IMx!m0#p|r6F-zOi9P*6a5=&KK=Pte)M|C;k$}*CN{h;3huBO{1$(+VKvXH zGmDdWF1e(9SN)UXoXgc~pE`G*+`0F{lU@9Zi))v2sz1r@NZ#)1hkVts0B za+UvHGmw7XG}%Od+5SnDR%a6#=Y{P^xVC)4ZWryPhY!7RvY&jgW%tA8n6Lhd7VkP( zg zkZaMYAp(*AVGR}Rb8BTKlG&i!$F`Z2JgC({04YmkktpJH$1N;@?h#g+SguZWyg zV)XZXb=gIBi;%76?N!UPuk}nlG)X2s{qu|uJT=ok+O-EbR?MCF#i(WZnqPmkXIPn* z>8<8Ge@Hyy$Y&7|p@}=MEIe{*!o;l{Q4_-t@wf@cwOsD`FBH1n;eO)$(wHPKX^Sn>kygalk?ws;kc4d`j z^@n|&_n)d>`JVCN*WmR7HI4z#?{glvu=O51R zKDvwK)z7)^qV~sQ(!Z9rZu@&;y_YO{pR9Y5Ve(JnUytPf7?0OI0oUiBSXb|<+WUlC zeb$Fy&p+8-hV@2;D(4R^5>E`>_jO*^-In8;Ce(U3#?CCxA^}aPP zpWIT~GiR!9M~3pkjOUvi#Xf{=w)&stwC?VvBjqA`cJ@7;epe!ew}0SRntSJn(M0}? zu36WFWt5+;J2zYKW`$^lX!*B}lNOKoqbCYp+}oga@L{&*A+1oOWYwz3I#W&kr?Vrp zJvUc&Ee_2!*>hORv~XR>l(LSpsJWAFa;Nx+T@v6uzV^sIj}u+Tw{y^{tm-_in#d~5HZ$P3;EkTIO5(qga~CJwTqvP*Q*y)786T(0 zE}yz%*3xx~rV*<;vv2-6dqd@!cG%$~UnjMDZJg_=mN&`f`h}cXPZfpNr|Rb))4eiZ zoPA}ezhdC&ZP~{~pE_(?7`?Ie>#lieq7UDNWjaf*fAq7u{YtfT`tgDi3lksLV@ITS zh8zDAn|UroIR8=J*&WRsijFr_Id{(#*qmXxF;H|1)0w-GE)v`uwo8BC5NEq4e9q#4 z@C9*Gu3V3wsWs)@?+;?r{>hzUy2Zx6Mo_rgvU}D?g|m5zNhDL9kefPTdx!?_&g%3LRRx?z}JiH^S_lIt>C{hJNZxS`%NE@W+gnj{Da5( ziSd@~9Xa_Mb8mJyu=ej!y>#fvbBn~NkK86U$DTA)Hcnn68o%W0Gl}!7YE3TAd*+kl zddv0JahFGXb#~89Z1UJsJn6B`RF1-+OTscP2G*I`a@o%hi`a!lmq_x)R!ls9WZC{p zD)UO;O>vGifBa*QwV1EV@5PGZb-iZctBljd_nAF*F#IiXvvo`ToVoAc+-SW&t8kLl zV~!iG_h(klnp5rd=8(BS+EW?zoga2@XxX-}*rA>8k5wJ>{?7Zyu7C9XBWL0GxYvr| zZsvd8cU#trmESVFymwN(MaOQ=rzQ40C)i`Zyj+++d6E5rX>C)}7w$^@7r7$Wg8#Gh zvINc8@XK4f)~W3>yI86l8~t+Eymi~!xE)q^N5=_mO5^0zXfop)oD@n6<;#+VQuaJD7X3qMIeI`H9ct&?O&!yfm=QeDt#H!{oNV zObh#?m@mw|w5K`6uzz{tQ&$NKhdP@gMkTWfP48QKc;4D(X3pF4jnQJprG3+rG?T77 ztcq_*JZ4kIDAXr&<`xgP*5w&fGr6@ci-nxqprE-aG4E89s!Muax${%msmjk2dyXcw zN$A&3J6Ik3hP!1KLtmS5!0#Hd)wAt&yOYqvJR)^SCY*lQ7=CyV?pEuTB@zW?>R z9~x!;X#SYM^?zQ;jBOKS%33xjJ{74^xjpe+&jX9geIUPphmO#136 z`a0O_n*ZvE`Hkl_BJ9_5o!k6jPSaOb(bwG{Tv=aF4_Tvsc;=i5=dHP)bMfzt?bx6x z{BO>wXP*tG&Q48#IQ5XLRVzpPA;t6uqEn}eKRm59alM#ckhoF|*YSCc;h{?Vx#EL6 zYt|M_Y!i4GmU2R}!dJCO+Be(}-SDrm>ubB$75lJ_|2$6qTCR0XUYq?# z{EA6`)xECl4_f#yQ&S{cpSq@rgsqq)k*+yOLVKfErmL?epPqQwm1m7} zf~}s!EOq=G%BCC8ULLsdVQi0gh{e7`$q&UXPG7YM7w-<9u6|d_%YXaS9bT3@q`Vua zG-t29_4{?#*MGlkR^{r%ZuQN&Y9_w=*4D~PyLW#1a}>rF8uyXvTFM@yDxkXbGcf@eHQ*dC9;jX_4kJO zUb}+SbJJclzna1!He1O3GtZKMm&}@T*o~sBo@}^vD`MldV^=LsCKdKQpZV3zYyMZa z&kp;xmz)r;RCuo=`)Qrc1mUI5^Ug3^Mp`|&Td80lVfAFCjc2X@OV8T%R+9EXIY#%| zzdsk76&g}*ALX@B^Zcsa{42NC?1<2nKWV(zF6!5pUlDRA{HO8#w5a2{k}maQiPY8$ z#=W0j8uQk4nNR#-QfH<5lH=jWZuVfWRb@U8)30udQi(s-_;-trvF>S~(?*`xp1Z8^ z{bOV2U8Q6<{dns6>F+j)U*Y_8G)eeL$v%^Rmv&D)rvKdVPtMD>`KOnExH45+Cti7i z^+%?Lx4PDTS9Zv(n0qNEV%o>7^A~c;N;WdJF5a(;JV}uFBQC6;*x9 z_Lfn+<*vEEip;ZbY;#J^ntC`r?(N#e$5>T%-dk&WC1c&Y2$KmSO*NA@oa**2t?li% z^3}_8`S|{6T#WY}-@M$|-n7>}DkZhe8S%RWFWayB@JMab52@|jX68&*iM_9sIBR1~ z#Hkm3k<14tX5Crx`ufRjVkW`4Yd0n&ZIQbE>KE^(m7>}~zPFviX8d_|^;X2hvhCL# zySBKwmDcV~xSD^&=->J;wqZ6>R(X{d*Icfdd!%Y|^zFdKGYhrFzO*+b|Jf`&Jz3l` z)BKk8HctIg%ddGa7tJWnWL$KAEJyt2M?%Y)ZSvyCykfyp-QLV2ux;fE(bCO`hB zVl|_7=p*Z{9Jcg4S#bkXV)>uE6ozud3HhcaCRm}yz4vHLocvUTDSN5T! z@g*Yr)jnFO-#PiD<>kgue~G-?E|I-y_uZKfCa<5T;V-msk(=?hV`~ym`%l~A^Zi>z z?v?(1p`W&EJ((|6DeLJrOD(K=Yl=|(LMa3BoFAzx4z0JiYvA)e+}78?;rsfkS^v-` zrVa%i{}#=}z_6B?fk6&s1Os(M9Wsbf8+;c&IrWl1GlR9%Mb@#~>dOE9N?bh}Tvs+e z?k!(9=bC|H;v|O0v2$NC|FDVs!LY9-|IrRhU7mu6zGlDQTE_3oFaP}e{rPx?0{LZv zbNp&1lw6Gx&}z9AwrBCJ-&!9xEA(Az-SWEK&PkE?uk+P+idUqicmG>>pXcw2`#JgV zSLI!s_5YCiw?`T$9G-u))h?~P zsqRTSJLhVx-0#ArQ&?txO1`FWc|H40zZ#K`*)Ntyp1LLQ?BC(ta;dA<3;E{ces>o? z6F6b+AEA2Q#9h+wO@+4#FFCupR-@r~+&O7|2i}ENzm}<4w&_Q3Ob%dg(R{bbWT$2V zQ-Mgr%Ls;(q3x?#%$^@LjFUS4X=PRG=T5_S&n6mP@o&;%_-9yndeg)7K>b*S|Tz1(b6>QxFXHC z!hcfz_1#jjv1zv2W@|Uid-`IwNPa@bxdg+|n?<`$nuR=)%)f3Q@oBE93=i7|x1~p2 zzBYNhU8xaW^wM_9M2nmoTRcPlOgW!z%CqAKKkC3*NTwdwTP6mEL+lI;$|%Vav^t?IgZIaV` zlZWg--uN3HlYbCzWOy!*TV!RUg5m6(Z?k6_Pk&$Y{7rv+{C?&Gx)RO?-B)GztBBv} za(SU0_Sa!rw~RYio!wg%{AX5c=iRwDDj?^xjrs{?m?bUkDqs5}nw-g9=x@~h{s8vIh) z!O&nI8grTT(!J;X^A)~z1?94tcm@V`KQj+>ZoL_xu(TmDQf1epj9FJMc4k@Z=`0L! z)?z80y~I3in?~Ys!^|YEtsMf6ww|vt5|3V}O6)gx@!vg9dP(k^$EJUq*d{I6m)5IKU~sQW~{Zj{s2#I*T>wPIoY$@EFS4azf1glY2(zFZt8D# zobO}5v1eYc?QvVXDzRUm{#04$X0)|)=9_iP*}q(LbEn0m1(k>T#9ne%?Q&?pWPI~{ zu`XAXS5VHnsI?x)jki3H$&&29 z_$eR%nbGm#<}vYR|F*WSZ|hD(D=+ry)p;)Fp};@k`AbFq3GH05Cx0sKxNzyPp7r7S zv^wL><=^+`F)m!}fVrjZmg>ta{{`Bob#x58c7IBVewFZ3DS=_dmw6{AtjpJ1-{5mdP~F#kamvGt z1&iT2yiFqlIIPDF)9V{F!@qg~*mc1JqIUFZ8ERhlwGGSXU z`cz9xibsfL<=0Ir+-rQx45#ybS$gAS^Z%y&eI+$7n@gh#UTwKLz3kZXf39=u(l@g( zzbG*(p7eR{`)|M3{rvDgzn;P4Kw7}%vYitQ*UhqQURZkT#H^z$n_@Q^m%U3dT>QL* zO*8rV=Ur1%Gd%6C?Z5ii@@`bua&OklQBnyRJ5r+3GAArtoBqA{y-suanRAmqJzJ6~ zx&72n*2M;k^*M^!Jqq_I_#9JTBeSg*n6jZirkmrs(vu4RO?~ZuE?wwa&?UBIV*Aq6%_j;BesZ?6v6Rdf zIrEA&_{Te@54x$pl{@_v{&y+fxV-zfqr>YE&M0x`gx{uHKioJmMWtl&?PHTS&C`B& zAv#rIuhhN+iK2-zi67Q*{r1%kjd*OSoFAd8XmN3O^puAu%yJK%l3Km8NOP-2*7UVG zg*B$hYo_?zo^;{xg+Si$g*j8t>=BgT8s7V;2wwF_${S)5du{w9mTWRB2}!yYSMh`vk~UUJ^mAjrTwn`QO#=^b^`EkX^%&W7%~ zuIhS>l~Jk=L75+Zzh>+I;Ir4UvbyszGo4hT01&D zg>0j2qNYp~YVzZF&h_h5SI6&@d%5!4zh6#E+Zk51-Ei%sOD2aG-jzHT_Sh^)<3+a6 zyMW}kul6WaTDyOpdgt-ckCkx;V*_4(PKi2vUFl+ZVZ6wRy|!MJVI?1(MK@O*x$sYO zyQ)rrzs{}JJ$_5??iGEu!`Yjq| zvaQqM3-7o8J}P`O{SI=t6ihhxH#AJf!TP};A+5*-t|Hr38~EDD{d(8uYF?00^e^A! z+}FRX|GiDV<|=%Wne<(C&%F6|$qRO;Es%C^W&9;`xbV6|*A;H&g%_Uso##Gr=*;Tv zFXr;^b*y3jJKN&#^Ykh0S*M(r{OMWzN9WwmQ`U2=J*!)r*Z8p?t}NMAqd5Ki{I{TKXM=};h%qlkSP!6p@Nei zpBC{t&!}i=`6HDqGAFxQacA7)du~P>#UHPfdZK8>`t?WlovfL+4xEWTbMAMYd&NEF zpb7K7?>n}a*Ty5?>HdLB=QhYJz8oBU?P=6k*ZRh7)=i-di;Skfd|j;{oe?0he$D#z z{GZw4>?1D(U=4_C3W-?$df&7b@58-x9gn@6_twYL9Rv(Sn|^j>L) zc}R4$>Mou3kB2vlzrI(dwtG|J&2kCRsJ3_V6BYU=Xi08*FQN7D@tSQFO>1JORXZ-w zeyCCWtZ>576MOAi=YJ^X&()3W%inuFtX)Ta)t0(vde0Q>Yx|Cz)A*yaP-^SjISZp* zEY^SEUXp$!YVy*?txTKlHS}-X(ut@26P(u75qBMOOQpxMrTxWPbj& zY;}3oFF)m-)5Ct8QOGa2{QPR#ZDp?cpNpSn-8|K?pM#U@X9(+O$7z$6RtNiE*FNk% zqrWUD&v)88>+kG7TG4!}C*<9jZz-K zR77Lbo{dvaP15$xJeB{*`6atKGxO#1e>B>5PbzpEakk}S#?!miweN0jPGA53-}CSL z8ALwzI9|JZ!1!lZYH{1;)VS?d((_*LUY4z2=5yFgBIvr#@lW{%XP%i{&K0$p7!tvq zb3cmfjrywFS(1KxP4*u(ShTD3kmb@yzWwrBvhJ*wFm0T3*RZU&Dc7R+P*raG?0qbq zi5BMjryevoma;_Acn+twSQyjsFS^$?wU*_USP2=qI0>qVw4Iz&rm$ulgos(FNGdADxoIWJV*Wp0+W<@UGYHgC(f zAH=*rH+$E5z1Wv-$;b<7xLHovYogQa+ zSDoE1$|-N)k^Q>dY-_6PoaFc^2BK$rA3Dbb-LqxL(_R@eojamZWr+ z<2<*_{j85pH;rc7?&3ain&q$H;nsg^RLlFXD<9f!d%i+1apIa^ZW|(&^KJa!VzWfm z#p&7nzKKumm78^Pv%WihUMU%KP~1qj{-l9z#MWTGIb6OfHD?PpJ$n}#vo*7#P4d>$ z7(d$;)%s3Hy>_TQ)>=1tXU;p5KWPzprp>c=s3uEomo!qmv+D8ZHip@!*2=MTU+45X zD$BU_jKPwZhyHb@|K)wxmAi&(fjg%k&-$CTlB^jne{z>AJhU^sSGITB!}ja8URQoz zPHE45(AZNtMe;`8{{4?cgrYz8vK_uD;n^d&{QtXe&W@4N9MfB(qF0>y!FONyslcu5 z$X28Lop$l}j4Pw>)N$HR5U&p2^+RZG#nO|qY^n=GdDa-jNcvbGdv8?1ZyJhDw_J7(Yd1;?Cr?a?SXLQpM--t)Yt*>N$Ss|MqCfi=UDxm*z+4B4MELA3O ze472uQT$hcFQ$vaE_T=|&uTKwV%6_%Syrc+IBo^0-q{Utxq;oUKg z-o+-n{oX59_3Q3AWX=6s;(xE2v&Y35iv$e1HKmp>Qt^3UB2YK`h~$Ph%vsUTs{))2 zL;jgA+dZSU&}T(=#r#K)7cn0TIN!1KL{R*rPe*FR<^MN2nf_J3boJr%AA7pje=Bre z-}hv9#I569ak9MO`>(EExhHPwqm$ZeN*BtzI=!735O`4SsN~EaVnq!P z>lGg%f5zX0OSFE;bHlp6nen}@9=cDyC(4D|T-BbZ#QG}Ha9+>lANwtQ@AMY(80@S9DIt8Vq*=O0-lttpMRU5X z|LsYa+IH|l_}e$9UZ47PyU6gcL8#f?WqK+h|Fso2N9m?alSn$fEZ1_{|2c}2&gL4s zaOchCoByYLT9bwI|Dp*?6B)X=M5PV~+${PrJ8bi@qtgN>i}Jmf`rVc%?dQDyO4-hr z+7dyAJig8%wS`{Ni5{#=UrQfayWPh=`BvL=O$pVmo+S&!+%`ScI}?-_eb_>8wZvi3 zHpYX!ztq0RRL{3?-@NlvWz?(Ug7uT5w^UA+N|ZP~t%@oAT$1YYbGN5Xi@zCJ9PM{k zdzAsQ+OxUE`J12STx@Q7%@cJtH-D$<@#nmH632G`JoIBy z+w2*Eh0Isd&y=s;w_(n)pL$k-hwdGkb46eK^!^`lr;Hl*bY7j`~<(c>ej0D-tS?+JVC9-tm@edQuo%!LWT(o1ckcYM< zpE#?fo}bO-y}ZGtt?`1TdLlN9S5-HOOnCP(SjBR;a_)nt_Qw>P)po@!c%b@Hq$6Qx z)zskVZC?Y+mL~MtiED<4bXPQniX1Rdv~`yc;wk*%ywIa(UeK1)^NX0ia&$I7X!Hzk zZ-~65sxr~u$*8^~o8hyLr2#HZ?<;vqO;Mv zg4cBZUbe9Ka;)APLl^V4Vm~E9ODBh0xwqfR%V{%_Le} zY+vN1yy)D;cMInk<;-Udo1b|+>f(hn>PL5FzkW7(!rUgU&l+L3m9!73X79clTlzim zuOm}to8ZX9~@UDnn@~p%q z>;lJmoSlkoq)%zrd(AaTGc<4dUNCi2!JB7h^@^{A7hJqP?eVGgfq@!r*3H*70w%6l z;lae(c7p%XQpH%e(4DohIXC`vILgimlw(EgqyzqHRdF`e7-vWeXh zcg>#1$*8ZV!7PH7rNN@I=_^tbSL4;T% z|LWeTnNLD9U+$RYo8^*Xf1GDF-(l{+~WDJ=i8%|TrFx_*hYQ_nlYoq`LKA8q*4VIlOzN2yit{)d2F zjA>KPM03|CmDv9hKlWYu`m6N?56xqKiH1*^e!}R_ov*VOsRu0od2zzNbld9t+asQS z*!Se@BjzaHiz_Ri)W$W}aQx0xUF;FpQc)YBrM5via`L>~>88G&++jl34{>xi3wgQR zc;&3|sLj9Ng|656O4Z_Ep|+)Im-j!O{HyNEFV2_mSQ1yOoSD_4Id8UJ&b{`|tFbz# zFMp0$Xd8IU|D=(?f%LyjJC1+2ypka=kYW38wg4`c4XwZH81B5EwPup$_mWaA(~a|yfkxH`=d7Qj*1lw}`u^5=Tc+wozMLc)@@I=^ zNT%wgdq>3&O1xq|vO)YzU0>3UU-n5qjMnV^?tI&N$z#3XYX7g9@0o4*`dZeXm*G_W z)hC$#r)T1Z3$mQr|2%%E^?biuS>gF;Dccj?lja|<9BHg)L@h9Q<9z;1U}j**Vn^S5 z6@*+yObxyt4BCdS=Q6D&#LIit(y%2e|O1j`IeV6&b+%>{oeBV?{nqn{=WZu|3A}=qsEOc zC)ZXkt4Rr6df+H;g>HqaW`)St-R;WZaq!EV4bqdgQj;zua`ZHJoSF<=7;yozp<+AV)zpK zxHWKv)VZ>SJGN?{ZQyLXZSy3qG~PML;qwyBwc_8NF+b){c08X#6Oc) ze&VyNj|YYSCPr~yD?H^QpKQ~&`iIK``zIEXqKCplINh>XPjAa^ytKwaUbK{VGN+s2 z;~&;@OlwxL+*q8#^wY<<+|w?;LYyzr=!BXDeQ3gF3uyo^(RYLUn=nHmtJ1FX5xX>mW`>W4^4`m z{?F~KX7sASa+XQad3)civuM0CZQg;UXS}K}&-(2$@3<3F{3E@IPe0EJe7eBz>x$*> zt>-6290}L@k$rEv*#6Y(+jy5h+I&g1Yw>prPVG8@<3@5^Uqt20&xdG!&`F;icmKg{ zmc3zz8dWnJCkDG_T(B%MeV%(T-6Tb|(^N$CZfT=qUwwyT?(Rg>DIDQhcYaJXyL&n9 z>W5n1*|Ltkf28Vqd>^pewEoKMSS)X7z#ndD=VV*|;c7eMKK4Vr&3O)+3QsI>yk#LI zk;QiI(kkQJU-~^9f9{F?%Z}b5)*64{Xj;Wwo}?|Gdkbv8bFE~4QlvjIYHwlB>Fl53 zJBk`M)=#LaYUxalIluqN)Z{1sTpkKAyU)m0>}!2*b!=n5PVT&KW_8PEeKiyNEZBWs zSuAe_OZUr!!nL!Hb|fwQaGvEmr(D9MvM#fJsk9$=ndbK%$ba0e_Cx>7f&=2#2hwjJ zNLJrb-t?ROLQ?yVIrfJZC)bGQ@_dt5y7oU>TzIyQ%%%;7$3MMG;Q#*PZqg)dvgxu?x&&V%YpR>7UJT8R@6^DWCv-_RZXC-32T*B{dVSMxp%lRlN+ zG@E6;z?~%wqT3uNHQZdxTl;WT|9|yYUVG3oF+(S-@kv$&245Zq1__k*t*1Y7E^ZCG z87>?y^G~mnhf`TWVbe|p!NZFTv$CTDR%;0HE)i8ZdhL>;q=H16LHDhrx3b>udw=Wg zMREC@>OPrMmbGu3|F4)=SDq%+rn2p%rE&SYzq$9nN2mS!_q^YpvE^fTL;ayM=|aCe zwAON8aY_2UnNxQyX9#WTe@7kumCU!X%-62l|4V7Kybk zyiuf=D%CuR>2>9ym`kE>1Xg%SGQB-`@3`xlB=KD-A-h?lv`%Yqc$mowXP%wGKXLvP zL;t6KUw&+4ujVZCc=ln&foluA4a|10P|tZ(eewBmzc1V-Yv=pxrJTtYTG~{5=A+Ts zNj18Kl}3wIJFW{)S*0lRA=&;A(^j)Rg_kcdbMYB=CT|QDU$k8E;j%{3Odz;k)VX=%pO=?ZHeOj(kCqW`jlBKNg#KIi6aJRP|}JwmRH zIYA{~MWtoK(T^HRmmWt(R!8}9x1_pGUCAZ$FMf*X8RJcrw$Jkf8pFa@?``8SJmo72uI`73uctc>zeI#)L7M#QaWM(S>n3yhdEV^8-OnAbfj zz7&|aBSyuSwQDw)L(F5RqdajFC0?xA_awlvNkS;oc=hTFskLf`$CDoT1a3@l_TQMR zHF5ESP4by%PM(_?{85O{BXjef0DdkGwwJL6o43qz?Vpmo;a;lol}B{8iaPa59QEL4evn(nV;ZZ@bUS8cx$K0k zw=3Ll?NL6xcu|M!oStsng%+{qAX68oKxa z_p*rnldfO7d7MXW+q>`|xBB%z_Bw>$EYa2FY)nX<`RQcV)Njcy=3BU4XB?{IUnQ7y zif8%OoUrr*yi1##?(RvPpt591)BHWLKW1_~@!$8r=0lU)Pm^A)bq(+A1jSbetWMCE z;_6P*vss<7_RxBf)lR1Cdqdd&n1r0(+Qqkz*;?oFG|epw=a$Q-|1Ud!R&>{w5BF3T zW#6#8yS*!~(Vb&mfucQRJ|u*Bv&y-9Zp#h3QDM4LO9KW7d5=p_2+^|sZz^ZFvbmED}= zT3zaMaz<%fdw0b|DZ6j=im{LVc3J&bj(uJCO6PWBl!XURdyM5Poo6qE;>vRFZ(PCt zt+Gwx+nO`yZN$5aHin#-ageorRrw3MRgdQ_>|>u6G~vI~zOv0%8|2=-f9S{WouWUh za^GCu+r4&i>@hCyw3*}+CZ^~p`HMLJx_eUE_Ul?j&DU!B2ejk4wJQ#*HT0XOX5`;A zS(AAFi&UPk__q782idiM+*st+w)R*q$I%Ci|A;reUs!FR!l(9bi{XW`wmS#kcl*ga z|I3ru^HDlg)?p7v0@r*C20L9Qi<7&b?4D|8FTSIjZTE+FP9N{JU;n&diQ24K%Piez zFETFPGg!Lxx|#Idk}vvKzID6*n8s#n@ed`z8i(PjwAkfzB$J1BcPhWi%y<&T9a>~PbM}P5OQxW=Ou8;*yhwDANa?iwVOAc!`ybF zM^4Z4sBoQwVcr*4tJyT1e|(VrgKZ7(<828h2P158(%N=l;FL!*XHEc zC$+nGO1N75+rDhG#G8M2+p8Zce7fwp$iP>BbvRR#?2+wxlLB{(Tc$GlEd6Iu6t-5BJYq4a$_E{6N&|_lf_ODuXHH+OW_x;q9_a1I}t?*jg zrt<8{ef~MyLKc=pZ|2XQBgNP0r}Tcw{8OvniKDix*9sRN`^n6}pvH}{;sW`c5ajxK z>D26SX;+Ek|DBm`#+aChuGpHeYM~mVRQ1Q8%!>|9Zbb)mdYzW;I_dX%+sV8;u@>K~ ze>&ZN5!f#m|9hd0-DcsWSvnhRpU!!6=G~dkzwNh6{`>j;`+kNy4Cx&!uT8xB@V49Z zIK5SOa=+|cyR$9T>iXQiYgZ-GjM_4HNVc86nRWb?XwEumzlECfqGOw9zpGkzRN&sG z5+{|_Z6{y5?={>0eap56zw|rrdD;^niX^v%Tvb{+Elv7RQi51{&e{oEL+@)Q#<)Fd*uQw-UtA3DrmgJE% zFY%oDBo4hmA=$iFJm#AxC;qINn7HCkwf_U7dmRr;V-_~JvnWL*1(>+s3W&M6@sG-c zV^=Ni9f{^yxctvdjpHpMiZYR0T1yt4idn^VZ{a7Q*}FNPsHpnuK7Da)v3p{{#%nKz1ki0m2GKXjr#LP$|AR(?Q~k4zV1Qo#c=m! zFY;c-#q5n=Z}N)ygRIt)m}Q+Yfd%0=B$jz^oPQ(aRkg^zEk7e}RGdC^EIM$*Nr6(H z=S!;1;}ke2|E+N_*t%$*$p^E&S?89emhN<`YumK5nZ?f8^~u9xuCEcVsx1 z{tDX|85AHszv`NyuI#=ZYev31TcWbJXqfs=WPM+CQ?OR_;?Boan>SqE{>x;=?%r6| z_%434_l9rwgv>j3mt*zWBHx3|jn_+JYc!^52=lkLf69vf6aQ}KYW>SEv-*S^zuj?F z@GoYW`yq78LgzoKg038i0+a8By*Pg4eR`EoV%t39d+h?b%tA$if*K}ng3dJwJKTRS z%h)r?fA6MG?2~>kv@d*^AM;0)dw+uAf9ZQ^KQd4BR94SV;*0IK*3V^+F+80x^F(0G z=in2+wAT1&y;ySKd|<@UTMiGoy_`LksLa<|v&QSiSxd(%0>_dA zU-ahxddYdO$tn}!D%1U&FMP1&wp$s!Y;(ceIh!|lcTVD}t>V5s3c?=M+?eET?IQ&Rl= zD8uksh~dB>6}6M+ASw9?9G;o z`Z;&bW2?5w^G--tMWttWPD{%yuKrn^n0YsGR7Z~LSMK0&8iUmu<=NFtdKEJ?e(bvg)-c0wPze#(uEWzMY{VD*Hu-gQ zp;W|@w#o-H&rI}}|1Q$^=gkcQn*ztx7&Btl!IG(-re+7Hc;6(ZN|s#+iVp!vez-#a}~}FVcOdH#Nyk}9p85D z-21zp`2llGTbhFYCWBu(k2(_nR+|X^-0q_#dFqD(f8`m;SKB;VH%Xew*iY%4ojK`u z@QPsj55JVzk~P$q#C>y>l8sY&tnvHc@mudZ*ZR4h+hDf-^6W`}+--zs*@Rxv=bn)m z5qR4E@&29CpT8X84NBVX-{Pt|<%{b&j*?%&d*iCb zrfs-$D0oeP7XNRxYZ5xMKTXK|nfAi6ze~3F{GxzL0Hm-D|akuHLtj>64sw+XsKT=SvIFKYiSkt z?O&Hlx>{pWcj<~+O0fjWNB4hP_Kxl8|BEwor^p_&Iq%CO5tdeLv@t&?iJ$#~KQFJ& z0fw0uvmBaVe%#*h>dQKD>D)z(VVsAKD4pdsxj2i<{fJq{J3hYe>o%U+P~V#V|HCh? z+1D$=56_p}@AvTf;!8_r{(8Ayc8~rCJ((|_f!bkJ_pY{7qsF^s{lDdWEDQ|CIT#pB zP~ttfq$o2l9qHT>`1F5nacYTYZb6O;d>ue;xUjoO9UD{fwNPd)t+m&-tv)M#BtdiS zLJfs2nbQL}6M}EwQk6V(>1^Je9GxGte;uq6vH7~?)7j!*i>s>A%~?*c`TyN8d8T#w zyV`lbr`7-a`#GN>(k3yW%kQ#ocjaXv9%t9{rf***#qMZJw<=5C5@@E=yzu&^O+3>} z{?90!y)FHkOIl%W@RYr(8|Njzm~WDMd3Mqc$?K>6bRTzh-8`Xlvh)WS&wR0j; zJI+{%G8MR)=LTk6yVkNhXN{-FV)kQJ2Qz0ST7Rp$d@YOj;nVC=rjvr97rQ+~xcrQc znDriaJ7)AI>$LoF%_L#}HLG46mSkNnm~0`hFgyQNYNo~Nsk)0VK9Ep}nzdx9;hgT* zvQd_qPh`$%TXZWXJ;MH{-O0m**@~k7?sR?$VbSP=4a)s)w_TwANnuBKdXK z*0Tk+>&h>kQ9Y!@W}2|!&YI|?p2y5>jkRHoXO@>p%4>=2?dLduvFEHs^rB5p4<9UO za0%o3`Lw`y%hV#Rb&n@&tW)$8yg%uy@eSS(or?Vl6;FFDub0O3nJir*t!|&Wn9FSf zXJhtsVfM-{4(}@=cU-i(3~RhnjA+cLrKT6!M#$7S2*wMRxe#co=Z#I~7h-J^9^q=F6QW%ego z{8m_VZsKyY-twKCi$1Syx_v=Y_jjjmmwN~gdry(kLDp6d-5{qeo02WRJ)D|oQ5UiI z@mpIR%^4086CY)~^m#S+NO9uHDfeGrTok#Lcm2yCUU!r8D}Kq&%wN7m<9fr@I`wHP znYJZ!y`O~MInK;#|G>Sid!O_6w4m=-Wc<(MPIZkflRJJsgz2sFR!#Far#`hQ?24Lx zjL%cz*x{`IkLPkRr&=m#UCEAJQEy!-AD)_sWzMA@ zn@=0fKXLKsM$ywweD~u_K1y#pnABX9b$PxJ`$AXywR8N8Unp7@Sk0_sG~L7i>n^VZ?3vNeBrI+y2f zT()iPN)IL1@61JZm*#HS=C@wunNybHU$$4;JyS02-tp1u=Ny>}-(?8+@rU12X6|9x)g?%RgEi4EUg3TDi~Qz08L{d~ZG2OYTE09BxT5xgk%8eB69a=NO8Ejln^!Lx zbO3hk#G84S0|eUAkG(E5o4usNb%M~Pplwm9;w!tN7I0_=t&Xy3crCXmSY~(89Iw61 znvZ6=Kd{{KfU!d3$d4thtHfh&eEMVj|Ihqm`|JMy;AF^tpj;7Xrz^B`ZBXZhdz0C- zHbp;4o%iJBE0xa65(hO?l&%_n|2uWJvu>NU`_faJ8J+$&ZLzcI>3?j{yE$;j=6a3E zMH>z#D+Mjq(Ob@%U&t*f&l9Tl|9#&nr&Q&)Ws9@g!WFhms*w>BP1~W~e@U3#=Gm?p zFCAnRXXIbIXS_30`N{&uJstWEYE6u4;bozXN4FNQEBopA*HwYx!|u-aIgKX2zy0K% z;5*Ij!wG|nmwe=+G7J)LGc12P`F6&F4<}YHj67Hw;l9~P^omHz`I$AJZu9ck>}i?C zCU%8$MOojit$K%f)-AXjK2z&ATgeBl-Rn4a$xZm#XnK7YWBfz4e|z6BmDL`Of5W*; zv|z%%wf@_u&wlcqRhKp6z0vwV8ZXXq-Tdrby=M0{+rC}>>HBLKw588Ub%|ar21KPR=K5-cT-BaXFtx2TD7)*?e(eIA9?rRD`U+qeiGn&${sao zi2g5JlElWqkj%%xpox++uG9o8x? z2x8vC=_I_hWO7cb>CL0Q0SC5yOTGVo``X;C`dP0nb1j$Om}d7v|7~Xd)Bi7D-r1e) zAk;OFOTGO0J!|9pXUfyxy}SSa&ll|mwHn<6qF38{mu!mn)n?igc=v?n$`zAdO9t^+ zE`JqO7Hy`vJe84e{;HLh^BY#}DhN7nBIV_E*39jOgxBh>Y1u2^3kOJD3VxCi8F6*W zyN5nbhE-2ligUj|WxKkj;S_5RYsDce#jhvZPU+}uR!weOBO5NX_~w$l3xA$CJ$NYm zDRZZx@2TF4a~5VD&(CXBzkb8`NKo#|{e@Z!&de!)J?oM1LotQ2uUq8#$_2Q+4;6ol zx)afrDl~0DjKfbIFaJj^lg@O05U4%1GSSkPCFDx$bL}lxj^B;wQ>Ul0RJV2JFhZbo0V{E!?a*yujJWwi7w)xNyrQoh0?EX*)Kr-h93DE|VAAtUDYAcVjaWFQ=KmGh6Nwx-;{Z>hnp>vTX~c zmA>f~ESj_5OuUJMbHm&ax5!3u0jFrm(B#EF3vXY}a$7oQ`nmPainhBmj`?=;Z8l|@ zv-_-V;FX#EvAni!UWQSRSZjkq@2E)d2HyU{u_Eq(dhmn2#rs=+3oM*-=2H2h=@y=8 zD>gW7kBMmAo%dzo{Lov>OxHyEm+WmmE+rnY*){wPAahuVjYt z$ZXDy@D&bMdhu+_)tOf(lqfcetbV&t@t(=oqg?7SM|cm~^*laPGofqW{>|SN_De1D zo1<{&W`%Q=nsW9^om3~$U_rf2hm9|&q$~)oY+w9YTFw8ari9+5Mbf8dUOpRY-gYyI zY58-}y{ReRzHlVZI<TrwQrPbSCRz*!TE#p1<{8-}JV1@oi zhT?JuEnoHL_?fM+&zWQEK26KWE^j73oYT2P@fj1UEb8p`rd|52*%7F;|ip0k8)$64Qw6t<|s zg=b7eYHHYGzD{F%BOLK=E4$1_`-ktoTm3t5snDcScJhin4ORP(-!^D9HTAFWwG~ML&@FN;=b+~ zcQ>m`6;~|v+~T)bO-guX1;6m~%Kv3t2WMQb<9)mTY4M!{pE)nh6Mb%U9V(+nGr7f#!c&hXo8 zbZM62u_K%=dy@`i8a;c`V_GqH&c@!l^AA}iyL%UxHoXvHo|LIwcw2Cq%}u_aw*0y| z(gh(m%6DnSESyl&q?c!&RvILP{i|)A={dadbozH*q`*HP_N9mLPE6KW* zvlf^C?S8(|YU-Yr+Pp?q*e;9eIbi#En?@1qI=52pvb8xlX zim%miPgtf#v7FVN7OruyMoq78CuiKQkC81~m(|)Gx>eJ%wPLEf#zWIRc@Mu%;!lfz z`+d*Jk2x*o;;-jAt=YQbfK}htQ(Mj^hek&T-8)(>c1Yl?1hd`#=YRj_Y*6x;y5e9~ z-xKrNClhOauBo}gq4wS`=E)xK9~&g*%j=&=7kpCmsQBlinz$`L_Jru%_Rlko?%Y(W zJL~Q8&zrV>{3`V8f6wc>bALCU&Jx~N;_)A?Z+tV@!D9t814A~(hCfe##G>f(qRf(1 zc$X*ywlsRGhSJv0IgLvkrap3CX&4|^>ao(9iOoDOTRl@cu4_+eNX|*+((KeqDg zUS7tU){uYZ!(xeZr~Dq>Qe$3xaov{h?<~L1+gtws*YD}~8Bz+u9rP=8?@xQ}=gZae zv|D`kw$RNiN}}2u&h#7zc{uOu+^Op#^Gsv8YL2f>%QW2^rcm2>vgz&B%lpc&7{0KuJy=ru0Cz^sIRdvr>~6_J@shgZ@1N9yWd&rabEwH zdy_-v{8a-*0~!Au$qOHv_?Ra%ao=rOmbr6Y_4cQ;zSpWxNON<(azI1raYy2*0|sta zuJ<^6nXzg|wg0WgHcQ#hT!m&K%6sn>HSaH{9>0C-`lq)$0(GS^Vz1I6ZM5x ze$e-eWY&z4a2Lw|Q1h*t`-R4djC|9K1+ODQ*>U`nP#2{7EZN}DX`u#ZyBgA5#54az!g^xg{&-E!w}zCDc{$ZLZWk z8QT+5`x0N&p31)Q_EpcMluuRN?^O0x+?al(z(RYKzH`U=AFmevIQ_$F)=zfS7OCIa z_)crk;tyU11~t?cDf(atxZeWly(|sO4i^p=xqIC~jk7sHM7~4Fdy6MS@P%th8lOUf zlUJ~;NQieXYP@)oC0#A#cisN`TmOpIW$fNnbLFa@{df1$uXkn|vKdXhF==Y$9n19h zo4;F@AG-hl&qwhF#y@8SPE9@KHgA5I>A4No>5{AV8SOpBdo;*}Ud1lSZ8738urjv~ySu9)`vpwizW!tjOpvHh(Myt+ke)>qrJZ-9K<87I(VWE=~ z{pY5%6)bb{`P{Xzi^p6yz+~rCy>*#9>aRYg>1n_A3JU%*bxClIR`T^ZXFjf8`9rDD zYP(zH_A5&Me)_S`Dzn@4y(f6?{cNy0E0FWH*ipQZiukBuvSGT-K)Oi=wQN8HJk$+-u zoW&1cY!5OwSCI<+GHGATwZ;5-P86(J=UDn>_f|Kj7snQM2`>`kw0&}<=;Q;h zx~YlbXId>{uYF$B!D#)}psz*X=;SM9>pm{ZNbHq0dgQ@!vL~)9>ZzCGf}?vM{VEJu zpelLG+&ttELuz!_s*@INf0f=??0mm6?2xfYcG27!Y{_z)Kc2``3_YKu(IRYd@)I>EUwdI1)A(OZTa-30CoE;2BNr|pHKL57YzKio; z&v|Uh0bAvQ_$5lBmv833-RK)=B(_20SGlK2NBnXft4VeBjn8`vciG>pFnl>n=+ER} zVdKZ#j2B*iesD~!fBL$QO_yI99u8RNed035TJ5Nr{`%eG-d1s^6`GSRTO8LIyx9}7 zyIp0)7pDG7n?kc+Qw6vdm}hQIT2|`a_O3(!qQ#1;dkvH7r{_HOiI+r|kSU;%!O}WDJ@I*3?Q?9Ma+_Ucr_IXd=wl##QH^R^7 z=Fb1DahEe^nO67MFPg7&wB%4q%_E6Fn!eL*Z`oC=>|elL{iU|-)b&LH&&|X9UR#-~ z9Z7qb;*kDVR`p@^EXMx$Cfb&>w{>5_ni_sOY?AWx;{qH+z6^bYGm}~Dzl+JIHD6(jX^s z_9h7hPvM<&D09xjS(nXjMV_jbdmhtyc#B>VQ|E`D_pEmm6i>di=Io~1ohvu6Q$feCf}l_hZ+)1k-Z)>G_La1gyInG|xpmD0GUg z=&5;Jsr>5>ZP4_M(0i(r*zsqbOU2z~Uj(0u#T>uT_&9W4Y3Pn!G3ylKz6+K9HFW(d zm{5Os(Y+5sU&9tJ;+woeIl7}t`uz?b?{zCYl>Wp`U9pe1%PdE@{i(v=**9XBY6K;A zhKRU+u#4q%m49$JYG42BPqksIv-m&0&7JYSSf73Wmy!$3tLMtx+1T0~UupT{>|+ZF z71xDqqCem4&i(T9THlXzZi!bK>yxADQJYT%tPVi*bX1)0R~Y0s*1h9SeUmsBP_9t>V_{C{z5RQT)L< z<^l&9i-HFSCbm1Bsd`(qpio5J`2YQL=gyt`^Y8Oxb%s05x-6%=@A2+F*u6nqVT#qk zOp7Yho<}RBzixAn-Tz$K;JUNM_3L+v|L?v3;%D>bHgW#L2XFE<-RtptdMNKoKF`xf zZpN>^|2_C}MH+*<0pIQO{&&K74;HRZD9aH@n3d3w+RV1g^?wda?*A7v?tR_5jQ4s% zR24VJ&cnjrEAF=~`ZOiud%F6&M&7=nGfD;d|6eS-s^Xv|Ivc43#Iq;9?nQOkgIiTsil*^Je|qn zK~o<6cQ;AkUHD;r(cA0K9nH-WnosTLo2?+v`1#WZ2X~d#F8eNa&44)!Q;ZMHWH^)Rkjk`S0%N!Eq&*W=cbYJ+NMi_!D+;l%?(Y0oq*yWNls$aJmv`dBVKfp85{h06M*q-Ap(QkzGGm}<@Bz*Ic z`z*HGtasNQL2m99r+*tp8lK;=Yd(|q`KK2*7Kjy}6q&K+(T0^zmWe7BaX)I3>U!VS zY;f=Lf}R)&??=~vG~Mzlo7o(CE9Yj4wn!Ln&jZtsm21KS)%R-kToqX^C%R{y%Gz3o zQ&o#{5^i1K%M0>~R=Zs`<6g~Ue%AfkAALAtzTn)qI%F=-rEi-|V*Nj;XU*y@+R%z=R3hE)!a|HOyYZyJbg;)FcrpA$5tWg?|^%3hp;@E7O_i z`QOyo#$o<)`%etDEBwBnXsb$Do@q4iqUHU2<>%l1TEG8)4KsteqC>%}w~zCjZF{1) z5?9SDTf8Ik^@_Y1Zf`9jJ*5)0T?+bzroKC_J5_~0&ee04LXYmdlrvtjdn5k8V0ieC zVd1U)?_|rPUWJuDul!-OX3L=+kuNJFBR1to_sU z|Ng4;k6vS6(OaX%qMW_b`hDm$$FFM)Q)9i04lA88x#d1}omI~yueuJdf==}p@y1)@ z=dE^&YS89*T4(+0y=LV(ryV?-8w#ejZ^_tXY#J;^rA>G;A&mL{c3WpoyetC)e(MBOJLtga)xwWkteV16}^*5aB@fEi<`aVg+45eAl%-j5O zm3wRT+cNLgjozQHA78O2;j$_J+T}89eqCKA-JSAvQ{fD~qsMce3Sa4cayjUM)YGR2 zmWlQ=pS5!D@{fF=_HFh(^Jz^EUAJex{v+V|(em>91hZtjwQ1L`S*+8~`E^OTnLVgW ztGlN4Z{?TXs|mhcyKPPPaxVF?OZ`H$Lr{*zAh=jCsJUKd*H18`is+T~fZHD+INM0&FB(Ii(@-SG5@ zZe1B`H4RSjiRaAr>0Ya}^5z^Jt%omn$2@GgX20#+^;vxJ z?Y?&c3m#89X+CqVdba_uoZg}EY*V%EGvC&UoO&PIUWc|sKXt)I@hi*>3^5!G42med zG>jeVkd!nxG`mSo&JsSXPRyj)+4P1=&l@sVj+S8nFk89^Ri8@i^SP1C&n zZfRcG%wNHvTQ-}?%Wpubf=VzmH51IDHSX*qUOV}}a<(FGJ|7TCm`F~OS zL-xLOtKC9+hAPe~rjM&R^CvBLy2LGWe@kyuhl6RswoV2Ay+@r58Wx|iHJp9qc13&B z#RC56w*HIDb&l;(zP&i(sIAGYrukdWJnZs`T4$6~*4?(_w4wUk@Co;>v(Gk(cbUd+ z;lA^sy8dn;*NEUwlx=dxli?9EA_MHp<5G$#= z$GvlX`>wcFaZkYwwaqowzd|-MHOozUb;5?AwQ2%Tsh7 z7i(?baYjgHwo%^0bsYCi+q|1)9Cj}Fb!%OQlr+zC=GvnZW6;|@JsR-61bz?`pQruPlS?`0lcH#_7iT7XJOme2eb z$-y5u?tj?#QFrQ`_iYywW^_;P<($>VCVoxxoOEAu#p*vxZRS2>`{EN?9P_;J!3tN- z{+FKmnXhKsrm4>0S^m35Ezv%fC4WBWn}bY;Mcyy^QYqmqw0Odcl%5rwQ9u2y+U^|t zQS4}NZ@F@tWt8RVigmr0cg7n|{LLGDp7pY{Nq^Vrm0yfP|COk0axZJkoO|}{{sU*2 zdjHJd@ke>dAO8vUjkD~Xp0Vo5C;pQxRNeLU!g1~st(PJMDt_wqyB6AaGrbi`D(@|G z&OB!Am$F^#rmyVcEynvN+hDt8yVq^nDx$GfX;-MjW45^4KblMx7f$N< zWv7$8`K(cPnfT-WPpv-!F01`(__yMib?_yVR(AE7zYUAe?Ja+IXZ7pu{S3P*4kz?K zt`s`KpQs|a&F*C8jBkqrR=vrU>hLfSIc7d&dWJis&q|U6vB3=I8s2(fR6?_p@K+Ys3k3nmk<*V7vC=sjrFE z4;J*y>eVWHIaMv$hzCC zr)UZHu^m1aCEz~!;4W7qnKk`SIYn3J7e8GZ)*wNpOyz*~!~#YsL4s)b_P=KQ{Qg)7xS4mbQqw#W|&Q$<>Jr z{T^axUHeibt#257_^Z0-!q4Vk>qTGKowT{>&%^6-OTl~n@-44Z><<_3(TK88_Y3#b)xRfhQ#;W(AzUg?DC+KU_?uaB_;Y)r}GWw-$>>n!kd|<4RkUv=XdHFQw(-$5td-=Vz{J;MF zcmMwW{r+5^LBmPz;0n|Jrd^w#R`E4O7T!+W`jn;Yh1uV4MZ5E=&6?!4%+g(7dgWiw zhk}(w3H2Mf=e;%O+Gu&dVRN$a?0gyF^HBcqZ5M zYUjUwpG6ZF#{2c(+Io7iUV-<=AJ)H3uB~^noF-&&{O;-R{FdvV9%e{6~HSK4ZGIrm}~Eagd~-Mvw4<(v9bCShP53j-9rX5?q*cCo9^{bvnG@iNL9e`fSSr`eH6+-cT^zrtBPP+-k|} z9v~R^(qz`+Cz^*%QhE9kWv=ddy*u&V5zccnuViYuf2vM1Nu5yN0 zE=^f3v$ydh*Q`%E?AfP`pUq27|L?mp#c=z{JKiGOJ2)O+(Onj|aK{s$%E*A!W!nN{ z>eZ&sSaHOquH3KOleAGWnj3& zgWmZ>TB(OLI6XBix?Ct-v4B#STkfPNlN$`3Nja;py6|#F1Tsl^%*u$CePArv zDrsc=BuW2U-PY~L_kBG(-EK>5$JTXaH?RL&v2NbZDG3s@T6EIv(#~x^Z}ZBpd6I9;Qy3SzuR&TZJIktxHs<>nd18G#y?*R1e5 z##cQ%dd=%@nYP?L`?N;C0;gdbN8!A~|2mvKO0MfvCi1g|hm5J(bjvwCTc$lIII5+5e%cw9y)QmLwER;1BFpQ*V*&Q{D||J)T04!{ z!&R#8_6zWpKV+3rd?uy6(#L#B-bsg+X|E>ztAA_Sve#^~9!Ez&D&HR6EW_qo0_^|U ze5{T?mWb@Rrs_J)Q{Da%+uFw#jtfFlXGYcL{I>O*9kZ-xvnHc{@q_6{z2-j6lU{S_ z;ghWvOfzbFxB20mP7Pt(OD{}gjJ%eAnL8;aZMDe3 zuImT)R>)-l@C)z?VZ&y?OI*9+bH;@8s1UL$hDeGxmEOxE#{5BGU26 z-xJF$?73zb{xQn_uq>(6YxbSGTWQPl(jIs%^N0~z9(?7JQRM?(?W?n0`ffS-x363F zb{WH@W!cNLUe+k;FUaVcIsKjB%7fe7?#{VBdCrOD>x17u5pYc16p?DwSy#s%YH63& z@-oECaO$aZoI9K&r+D2oneLvZeeKXGS8nZq1}ElKT}6>A;|!VhT<**MAq*b8Y&NDg~C^d_BhIw%(@`$clo2- zJL$_G|1Dct_pI~V!ii6mUrjiwv-*V2na9FL&qMSqCwpG(UTA%}q(8y_@bkEuE`5z< zYOkz69d|dn6VBf)V1H0W1Te_PbC9l+0eO~wPeu3J4*2lMxuZesBH?S-!*Z?7y~vUBP~sou#4H3Yct zNl0yTxSz#p>ngB(HW!D919LAE)03{~G{!}jS4`bzomHsy;D{LOl$FxDdeIwVo;lb% zy(GCYvl|yDOlz%|ub^+u~&=n}YY<^855P zXy3lv=g~z%UYlEr49>?zy)PDdm3DXv|C>UYe%9A-eJoA?h=X}b#sZ_!p{;LcR2hs-(zr3qf91#=FuqE z(hK*QAAdp{#%5!YoF>P{z!1fQdGS_3P9^e6wy9yii-ki){^fb@K3ld>*S}dqr0JWZ z=2Dx>K?@fiJz%C=+u$0o{pGgE8*iqaW$W(xapXJmeOA$$Fac33KaDcC(-w=3|2^xm z|GeB{-tOH?d}kR=@+mX_KIiw`_jS+j6n|e|U;k6o!SHcry&HPBgkhFN&x28N|d z&ldMAJaV*WR-2T_h6#=t?{D1I-ad)v^Ha%fZk*j4kMz4noZ4{4G(z+6HKD9n_y2*W zY2_=MncgWb-5WW}ui><$N@djn$H<4r_wMNblJ}^}{P~8GgB}H(+8e!G7(`u;w!Tnl zyyDmtd-(DM(`O3nzbbdIcQmElVAoBph}bGIE!-_nyL@)Rbg}Nqb|%Y(LY2<0R1>O< zWm%J~vEi)F482Dy*_7RU{8hIKYaLwUKDQ$(Ct<~@lgS1vB$gS@FlxGK$$yRe#)*L0 z>K}_cUVCoxbYV72|M9FqaKqFfi;SpMLfzI=-|X7Dv`1q7tSa_YYbT{`3k@f)$uo8g zE)VEky=&HkP-|^PA-O|-_Dhq))26La^<1!_BW>Q|O3&M;=Die^nfWEb);ZzaRKqV% zw6CnG%Ad5n+3aaVns08|KAwMhxhDeNOmUrk?!Nur`(DK^;urjWx4B3^m_2*p8l&{h zo-Z8IITzjVXSKbov|#o%^Ch=p-mI$G^z~6ghn!5xjZgMsGt)I)%su^go!VK+y<-3D zwu76!HD%&6|5ftk}i3Y-x{NZ}^4eC-?IEzRZZTPl`CT{p=b$ z*G)Ezu16lfNWW1dz9;Ye^CK+Yy(>dL^N3bTmE~>LIqloqp!mx1)yFA|)Stg_Y*)#! z&{FD4=E-q!YI?p&(oJlRcjN(s_rd3imwjKe|L~=>GO6<_2c0H;S6;P@bzS(8Fx|Qg z4hzSe+{x4JC5^wH{C|0m;2fQUlPkK+7Ff$IX+CRmXLUn*M*4B~rmsTto@!gPUcT}( zxNY*Pf|!lZBI+X!eXeL+{?e6cLn+?}Bfp{s<bp3IBV-64RWR5SrkN>@nkd-~&yw4;nY)PPfXWPD4 zw_e|vkJ@$Rp$`aHPmH^GlJ4^Ma2u8hFW8h>^ z%~>PQ+GoKsZPMHZBNixfF37C)Jv`^o#|iomxJ83*F}G$e$$OC577)1Lg_>XQ zqml&~T+5xD96TQHDdI`a)FfhSPiJ=#A6fDOSzRT=spI zV3n2g_stncSa-#{2|vs^{(j2Y{U;yGf0}#k+qC7m9M3=4W=4Bx8L7K5uCti+|G||Y z#)bb|R-@Q%&(wLB@U}(^6t;QYZJH#^;~cWoOVU(#O=JPX4nA(a0}BJvqe40K|Kz;B{Q39K zzrWZD+Bpwy`IlO9cayx8)729HzwZ*iRKNdIu)DZy1Ityn+~1;!&;BfvQx2K0$?2$Z z{oB=R&9!k?zxTgB_g2-&>7Tp4%9Z`j%85*qoSGI+R2FOQn>&%^F4wM;A|HPrzpbva z`m_7t&IMmGk~Nm9%urjw5>gquX`jXkiF4{INixo?of3YZmp*(WvhK;tH44{a3@WP|$JvbMM|b zckAONi-o1PcCEeZy^WE@DO0LL;e^VGKwj5|fG4Mq<;;Gw?6s+DOtZ+xGyX5$KVVdp z`x&@mbNcnZ*IR<(%gXkBK0C+KxcJ$h@BicLnGZGWpVd?{~D%O?a^F@dQ`gL% zSpIGP-^!dUzCAnU%BD3Mt9|F%bHmrsFQ8HKY=RKqp-VH4HRkp5@$EVB+Wvu8h2ZAr zGi_yU4?Pn3+ghyn;_thWU;yf%X!=jz1Cfc z-xul>I^(FPm-h5ar@t0I_nq-;$F5g}5~`1O7f7hTIrdYkNPOF5ts-&7Cshw8o@V^z zy#1!p1GVo@bKdgo=ne4U*5^I`VO3uN_dABB)3xtc$QZ0x^Ky+~#u{k_hKN}Sz8@^F z6;GI6!T4DVmeH1EyVSG+(lZp*tun!ShsVMewQa0?$)f$D$ig2gRNI5x!9E z>DDv%zn*!=b|uwEV?PTo!_#eD`(HA>{BnNs{eucw+jN3LuB1%Txf1Yjisps|4I6Te zxGT5l%sRbr){--aY*u+P`Oezz;cuK8ym8L)Nv+%~PMXYmG9@EVYr4~JhUFVNs?PHN z*%5Y(>$c9!Eip=8x!qa{1^?OpWY$}#v!*$8cGx-_0mo95X z_i#UM+p=o$8!mR~H(!@|ZueSZ!dzCx{?F2~HT2A`n^|ep7ON$$7JtA0FunyLo^5 zX0UtWn=JPi{6gW@D}FFm|B5ksb>e!s1pm@cs%ho3U-B-jpL#)h#h2aMPQTeE9X`Xy zwXl}k)57?%TgLARDPME^IM*Kb5I?_cMzifY*ZV5#U5IN@(mxk={F&9JWhA-=!19$Bj!zv4M+#@}nSAS9WIfuWs+fk6@_<3lG7i=q3t zb0^n=4jnx<|Man2Q~RdBV-;{bs1+@$wP;1diZhvF+%tIp?4OdRc{6OY=;QWJ{GDbJ zzJD0%G#>BV&U@^Vp5=1?->-|$?fm}k`+WKR3~LgnJ7)e)XFgv0wluaa>9p;~w)bpx z53K9TOJCfGot?NvHZ}A8gR2v-UXVOB?eX_BjO}N4-G6=a@8pI1jw<9^$F6_RYRxk< z>#eQxws-Rj?u*1Y%~Ra2TOgV|K|~`!(rt}dbK2Zn^0}L{zgwqSOH};j+nGM=cRSA- zKiRjkk?p>l?%QsVfop|WeuXq0<-`|jxxpln8Vw+O3n=7ku z<*E{2w@6n@q0f?0eY>^1W<2_&HhouDl1hGI=86kSmF=f}MJG>O6L_;p>GS#H%G*WT zPS(GbmzlrIPvKxlf&8j29XHD_4GTloF71n4d$A+Zc7wIkg*lg#)O|McPn`d*yhe1 zR$0+|wuik>%y_DNK`5gpZepgY%e6-_J;j>mp3Yd^u|}a^Mx|)7r>N#%I;wON(k1-EBs&uU9G z_!4Hd`qQ30-!_GK82e1~S-x<$(5nAWC&X!oe0u5H_F_^dXLLosq|?kdV$K%tue|tn z#r}@S5eX&leY4x_G_*o8c$ifaKZ~z*o4R|pTj(*19m#JbcctDa-5YcF;j-lk4o4MC zcoV-IJCWO=_>z77`rUqOQ>!vB8_n`%J?t`Zn#hzR0n19bHqW?w$)Jl{%dKGI!t+nv za+}R%v->UDjc;CxjPlx2zx{FahKgpRM2V9kyP1~z=eX`ZcKhkH@OyEkQDti*GiKLZ z7w%j8KC>O}{?hAuxlTUxhzG9Q*djZBR8!e}jJ3j;>76 zaED4a+vSgT9$Gj5?4o4VH>PJ!{Yp*w@qGX3Obhg>)3E^v(sm@XP#kzXq)U?eATJ$9e&t&*v4szu5(r+@3Idht5@n*FDtkLy*>ud|Z%Qy$;l;*iU{`0~qTH}_V*vwVK1kg)~VvJ>itpc|$zIx-{wm9RScH^%#lP4ZqyFcY=%yAKEu5*(Y-FiK> zw*MW6Tj-|8N}^j99n<8#uJh?f&)&n_t5re|N@PZ!T^26EW_+AQ)znAfPG6zR3M=a> zr&S*nm&;a8j0-53`DJgB&-Cn$0-g;*2bw-|f{> z%L@Isov58{}l&|NX-&-!tAQJ6*K>;_}__ zOP}BD3vR3TFwZ)Y81-9gdjVTIP?sFk8^a-^X1oO98vRTIirO*qJAZ3lo(5`rfo_lEvXKYUiUo zAFkbgFY)bDm9vu+3+UF129Os@u%cGi{3=Hx@=<_`2i(DXGp1I+_qeZSu z)}6l@`RovrwvJmwNY9M}dL154qCBAjI*Edh7nN-lY`@DGR-Rs1e$BgzQ|i{O->Q2r zeo4BuY@6pT32DdV+%40eRDLSo^53!k-R1XZ&b%v%^o_14OFL(I|EJ#ja_i@d>-T?8 zXNdhV!Ld8-)6E@*b9^LuO5J;1jLKDJZ#4Sola;QwZh1*t7k8Cb{u2o;}u>mbm5W_zTv-yDTqDZr$Wrylnk^jVl>N$F8icnHEufPI9_r)^%Z{6CZu7*3Raf zv_(HtZEaoK-L8UGwOxwoc9i3yAeKN;v_C{A{;cb(GWwcLfch=m|`f$T7VVhf+ z?#|!avMYp+L~cr({AA0r6N}fr7nVCI(Eqr~OmjmKqx3)dhx)JU9}2zqS+siT>2K*L z!~gTX-!_@2@!Gbp+=rsyN>wgvJ{TV*JKrk%h_2*ZmbHa;>z5a0+dL5vh?hV5H;;Gs z1>HNjo2%LucBM=;5qbB2NyN>=0dIZ!BizColowAwIQ{bFV*#f;q?friz2dqi+L2@U z`$LZEFNce7cB&XGc%JwBrmxR)ndv)iTGlD&zTTR8BW{+oq42S#ANQEr*-kh)yYtS4 zr8^AMRgIVRD0UUPH|N~jGAVO$u|o056S8M#3!6+R>X8FU*5~%@edYX6gCo?UNwOErF!D1$@ShtRqeUoRm(Cy2&+$M*AT3LwR8m*O8F9I-C*niJ>e|%pGMT@XT2q#s>00~w zd1wt^{DL=f)?zayHqNzwaoBst^^AX9uG?j^n>Sa5^+z9S@m<@jy?E^or#o@SnmV@Y z2i}XC>0#u)Ga&YitD)Mg^$%v9Sh8NoYv+^hHvV|`kR!*;U1a8KO@2})`EOd==^tjC z^&%lg`@Dme8{gdG7qoqyPGBncdF>zj+TK58Uvlig#uF>}?Td8&eO%=g^5C!Vqqi>G z`0ZUo4zK=*1oKKDPd(a%OE_ ztLNcLV?8a`@cU9J_07^!amVvDyDvTGy7`ao7VDk-pqwd;;m6*eytmcX``GQ2_pZNZ zFI*k-|A$1${ex@0?i`Wt?9B{_T(axbW6i2L!u{MPU)Elkn|S-wq`w`)ClY49`}KcC zng5qZ_JUqzGZ!TDW*R5Waw`icNiH;0+dIv} zqVz$?C)ef2|H$56@yM33Q{X4}zc&6AidVagIXUF4b`@4+`YadywNCW8cvbktS#4fX zpD$|o=6Bwk?fmfe4S}rvy}JUA=U!<)bUK6QuA}kavOUS0S1n36HDs6a$WUhblD9PQ zL}G*g1I+{fgqmjjRdp%;?XB^&aB9})>s?vCCQhy48E$9#%UY&w*c{$f%W=Px?Im-_ z{R8vZGCBSAlftdne*IE4*L3~fYgZm@z0Q;vb-P-8{-cN%ox2TBDp#*4XkB~4%hkQR zt1oGP@01@_545bh7=L}b@J4}e7gt)Q&Y5@ocDKK@ojv1kA*GuAG+wvT5Di?@6 z^uwrckABFm#;k>3c~2bLFgvyMRz~ql{a07~t!({Y^SBF^J~h9Ud@EM*P15QL1D^$V zmvZkanRn~4`DuYmmB&e;;oA*1jUrrTjPZ6`6Mmf2)_W zoh%lg|MbJ2Hs6J8oodxDwiin-mt7*-@9_KM3Tv@Gaa|d8ZuMN>^CG%AYEm1WceZ98 zdX-~oeO`2>ciFO2f_bYX)^|^y zBy1O-yrVlmXuFkjT#w#kYwsV4PXwjKyQRzZ9zIw7wXuKc`$*UN9U8yn)*W{`{o|@o z-F5BOLrMFKeiGl#OwIN~Z_5kz*VLI>tGHID zcG)erI639B(Ay6dJ6->m-27_d^(LwH)Q>}d{`{$nJ9u5z*Yw@Q8Q0&5NPRZq|JrM% z|K!xGhWXlAqK~z%TTGA5n)P|*p03Ya%UA!X^vxChoSnn``BrG$)6%-Qor{Mximns-3%%H{?0&2!g2Go2l}@9250Ihz-qn!Wua*K);7ll2<6 z3%}m_TB&(Lq&$D~qBj~Jgs11aR<3eTx;SCl?Kw{ciuJ#I)p_&%$`%A+PB6q z{l&D2FT$tzrKit0$(OqM(c42zWyar|ZpH?Fvo2-ZsKaPp8~`uXCVuSXZp>-|;v3HU{r@qx~dA%V84-%dYDd z2{yd{|Gn0+;Y&OCCZO1Ezu=N*`}XbQTUQ@c(|t2XZr=R4>k=M{{cP!Cx+ozpS0T5i zLQmx9l%p#Ydi(^Q^A>6}?YG`HBlb*_^h6mqg`}OSoagM4^b)>%qBwWhE(9Tm5*{^naUJ;clM(JzTLOTU(NKZyKrX2WeGhhXky zDk&<*Lar=08_=)S+im(SPV;%?@~VF;zh{-clH6Xw{Zb>BbMu-fktwaypZ@fUtN8Wp z>&f}6h2wRn{gHCWP+Y;zrhVK}cuB$@!y?~bL8sL`RaSm>V7+Oi(z*KVOqJ^Nj)3>I zd#fb&Zw(jT#m!zMeS|^l?VhHMK0lfk#jJ@`-1Q@@>uqz&XOU~I+l}r_W{6G_N#Hy- z?Yr**^@De%RgX)$J$NTL;o6SAhc?;|-|Tzu=AL~huq%+q{r>ft9>#2%$(e1*9+r$t z#NETWf4$Xq-niH*KqbO`cAdv!kLYE;te+)igg2auV+IGrhU&)#HXq zN3Mr$@ON5zT&l_3%kby|1GAb3ZocJ}!EHtlFY#QRxRB%BsSW8bH~iO;FFRs-Z>|3^ zwzmQE=Wl&-ua<4kn&nkKnS!qBDtoH?IW=>__q zTRA(qBq>YqUO6xA=WsIhhR-eUB+rw98{>{%*yw6>vT=6oqwVsa{fvZFkB3iiJ8o;b zMss0dnA%(h(v-XU*no-cGwmDh9X?|k=(GLkmLFwR^Eh1{KKrj>_xsg7%hZ2-g_`VHwx)T(Z}z=^l%0I>`leUEs=ll* zR{Wi%mmzOmxZ!=KT1l^^W`U6Co+(c^%JsXhTcz%fe?-QmtJ{-bG#1~=;%LvZL9zIoMFi1{n<<2mkOwWK5q>fj)wz6%9C3#{3M%e!BP)Le^@HT`NBRJX5>?;p}_&7k_wIPjr& zA_D_MIs*fPF)0;IK~a7|YEemMYO(5xgIo>{0t^>^*BzZ={8pp2Xj-$dy+K*~wi{R9 z1pS*=RK0gi`gx~LT^;_OlFi{aYd*REUtOD*Fe6RFG^=#8i+D{t>nh(vrF`3r zA8NF$?e0?9Io0Qvk<=~C|G!xSycwB9n7P0;H^|Lk29)50(hLj&3~wDl9C(e6(8j^w z5}#yKns(+&1p@=aOa=x95s*qKUec(J>~UX5KTkK;;1E4uH-rfc49xMUjRzK}ZivMf zp|ex?bU-`+(gt$sXQ&QX?GN)J2iPXAZENe02V6nAKzK=`G$Zl^IPT$BBR%lq0EB}e z23y)$Pg=pkz_5^$fk7W+A`~xatYbkn7_@Q?KBAioyBaho6(kNl2Nd1#{K{J#pwPX* zz`y`%_(08OSkhR`iE4OCX0dZlW@=uE3h3xJ|Du%CBJA!hi!X2!XJ%m7$hbw%nZB#j0_CFm?7H`VWurzJ#;V5_crR_XJ%md z%gVqY1v3Hco~!)Gh9DMaV|UD%cV=$qnHd;dIUtwK!HincC?kw)j$eLCD#9#$E)zBO zI~Bmh!0-op?>fRnU2$X+gRncuAba1BP5}l6ISa@lMuhe?k|^3i3ojsXhuxq&ya=N%nAKKhF|e+ow?M8B{T zVK9>ovcbqj2jcopbRVN%iij}njUBRSs2g$crBhzCThkCG$~q&Ph;oM-y0g%4Z9*9G z$PL*LPn0{H&`m+V`~zXiLmy;QkgozkHv|2u3WOQue#mB^Bu;$cgMNZN!oZC|$OfXM zG3YEay5rCf`9_#^D-_u*hzIex3jNqtv$%5z*cjj>qlP(hR|O(0a-sZ z85QFw4AIx@AdHcijBE^Y%pxN)z1`U8BH2=fYj658Q+ZaT@0OSVK#Taft7=t=I z3NsS>04c;}pngFGu|`6NOOb;g)VoCQ5hF}~xROZ6f=ovAj|03}*+7OEF&HuI=VM@) Iyb8nv0Q<{W6aWAK literal 0 HcmV?d00001 diff --git a/embedded-repo/com/redis/redisclient/1.0.1/redisclient-1.0.1.pom b/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom similarity index 89% rename from embedded-repo/com/redis/redisclient/1.0.1/redisclient-1.0.1.pom rename to embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom index f247482f72..e1cb72559f 100755 --- a/embedded-repo/com/redis/redisclient/1.0.1/redisclient-1.0.1.pom +++ b/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom @@ -3,6 +3,6 @@ 4.0.0 com.redis redisclient - 1.0.1 + 1.1-SNAPSHOT jar - \ No newline at end of file + From 36a9665828cabcafe98294ee9b1474ecaa0fcab3 Mon Sep 17 00:00:00 2001 From: Debasish Ghosh Date: Tue, 2 Mar 2010 12:59:02 +0530 Subject: [PATCH 08/15] upgraded version in pom to 1.1 --- embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom b/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom index e1cb72559f..16dd81402a 100755 --- a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom +++ b/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom @@ -3,6 +3,6 @@ 4.0.0 com.redis redisclient - 1.1-SNAPSHOT + 1.1 jar From d46504f72da2b0d0587aaf56b76965f7d253fb12 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Wed, 3 Mar 2010 12:46:19 +0100 Subject: [PATCH 09/15] Had to revert back to synchronizing on actor when processing mailbox in dispatcher --- .../ExecutorBasedEventDrivenDispatcher.scala | 19 ++++++------------- akka-patterns/src/test/scala/AgentTest.scala | 2 +- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala index d35b8205d1..e115800d4b 100644 --- a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala @@ -62,22 +62,15 @@ class ExecutorBasedEventDrivenDispatcher(_name: String) extends MessageDispatche def dispatch(invocation: MessageInvocation) = if (active) { executor.execute(new Runnable() { def run = { - var messageInvocation = invocation.receiver._mailbox.poll - while (messageInvocation != null) { - messageInvocation.invoke - messageInvocation = invocation.receiver._mailbox.poll - } - } - /* invocation.receiver.synchronized { - val messages = invocation.receiver._mailbox.iterator - while (messages.hasNext) { - messages.next.invoke - messages.remove + invocation.receiver.synchronized { + var messageInvocation = invocation.receiver._mailbox.poll + while (messageInvocation != null) { + messageInvocation.invoke + messageInvocation = invocation.receiver._mailbox.poll } } } - */ - }) + }) } else throw new IllegalStateException("Can't submit invocations to dispatcher since it's not started") def start = if (!active) { diff --git a/akka-patterns/src/test/scala/AgentTest.scala b/akka-patterns/src/test/scala/AgentTest.scala index 17ccce8e0a..a415d9c218 100644 --- a/akka-patterns/src/test/scala/AgentTest.scala +++ b/akka-patterns/src/test/scala/AgentTest.scala @@ -21,4 +21,4 @@ class AgentTest extends junit.framework.TestCase with Suite with MustMatchers wi } } }) -} \ No newline at end of file +} From 259b6c21bf786c7890414e45aafd8b1a94c87b39 Mon Sep 17 00:00:00 2001 From: Eckart Hertzler Date: Wed, 3 Mar 2010 20:06:06 +0100 Subject: [PATCH 10/15] shutdown (and unbind) Remote Server even if the remoteServerThread is not alive --- .../remote/BootableRemoteActorService.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/akka-core/src/main/scala/remote/BootableRemoteActorService.scala b/akka-core/src/main/scala/remote/BootableRemoteActorService.scala index 429fdb61ec..1c31c3025c 100644 --- a/akka-core/src/main/scala/remote/BootableRemoteActorService.scala +++ b/akka-core/src/main/scala/remote/BootableRemoteActorService.scala @@ -35,14 +35,21 @@ trait BootableRemoteActorService extends Bootable with Logging { super.onLoad } - + abstract override def onUnload = { - super.onUnload - if (remoteServerThread.isAlive) { - log.info("Shutting down Remote Actors Service") - RemoteNode.shutdown - remoteServerThread.join(1000) - } + super.onUnload + + log.info("Shutting down Remote Actors Service") + + RemoteNode.shutdown + + if (remoteServerThread.isAlive) + remoteServerThread.join(1000) + + log.info("Shutting down Cluster") Cluster.shutdown + + log.info("Remote Actors Service has been shut down") } + } \ No newline at end of file From 36c0266a5d51be0b6f1189b706fbf2a4b1141552 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Thu, 4 Mar 2010 19:02:23 +0100 Subject: [PATCH 11/15] Redis tests now passes with new STM + misc minor changes to Cluster --- akka-core/src/main/scala/actor/Actor.scala | 4 +- .../actor/BootableActorLoaderService.scala | 4 +- .../remote/BootableRemoteActorService.scala | 1 - akka-core/src/main/scala/remote/Cluster.scala | 33 +++--- akka-kernel/src/main/scala/Kernel.scala | 2 +- akka-patterns/src/main/scala/Patterns.scala | 62 ++++++----- .../src/main/scala/Storage.scala | 5 +- .../src/main/scala/RedisStorageBackend.scala | 103 ++++++++++-------- .../test/scala/RedisPersistentActorSpec.scala | 6 +- akka.iml | 6 + config/akka-reference.conf | 1 + 11 files changed, 131 insertions(+), 96 deletions(-) diff --git a/akka-core/src/main/scala/actor/Actor.scala b/akka-core/src/main/scala/actor/Actor.scala index df536fb57e..a6b46d903d 100644 --- a/akka-core/src/main/scala/actor/Actor.scala +++ b/akka-core/src/main/scala/actor/Actor.scala @@ -309,9 +309,9 @@ trait Actor extends TransactionManagement { * If 'trapExit' is set for the actor to act as supervisor, then a faultHandler must be defined. * Can be one of: *
-   *  AllForOneStrategy(maxNrOfRetries: Int, withinTimeRange: Int)
+   *  faultHandler = Some(AllForOneStrategy(maxNrOfRetries, withinTimeRange))
    *
-   *  OneForOneStrategy(maxNrOfRetries: Int, withinTimeRange: Int)
+   *  faultHandler = Some(OneForOneStrategy(maxNrOfRetries, withinTimeRange))
    * 
*/ protected var faultHandler: Option[FaultHandlingStrategy] = None diff --git a/akka-core/src/main/scala/actor/BootableActorLoaderService.scala b/akka-core/src/main/scala/actor/BootableActorLoaderService.scala index 1bacbf6f59..0c84d0965a 100644 --- a/akka-core/src/main/scala/actor/BootableActorLoaderService.scala +++ b/akka-core/src/main/scala/actor/BootableActorLoaderService.scala @@ -31,10 +31,10 @@ trait BootableActorLoaderService extends Bootable with Logging { val toDeploy = for (f <- DEPLOY_DIR.listFiles().toArray.toList.asInstanceOf[List[File]]) yield f.toURL log.info("Deploying applications from [%s]: [%s]", DEPLOY, toDeploy.toArray.toList) new URLClassLoader(toDeploy.toArray, ClassLoader.getSystemClassLoader) - } else if (getClass.getClassLoader.getResourceAsStream("akka.conf") ne null) { + } else if (getClass.getClassLoader.getResourceAsStream("aop.xml") ne null) { getClass.getClassLoader } else throw new IllegalStateException( - "AKKA_HOME is not defined and no 'akka.conf' can be found on the classpath, aborting") + "AKKA_HOME is not defined and akka-.jar can not be found on the classpath; aborting...") ) } diff --git a/akka-core/src/main/scala/remote/BootableRemoteActorService.scala b/akka-core/src/main/scala/remote/BootableRemoteActorService.scala index 429fdb61ec..1b56dcea5a 100644 --- a/akka-core/src/main/scala/remote/BootableRemoteActorService.scala +++ b/akka-core/src/main/scala/remote/BootableRemoteActorService.scala @@ -24,7 +24,6 @@ trait BootableRemoteActorService extends Bootable with Logging { abstract override def onLoad = { if(config.getBool("akka.remote.server.service", true)){ - log.info("Starting up Cluster Service") Cluster.start super.onLoad //Initialize BootableActorLoaderService before remote service log.info("Initializing Remote Actors Service...") diff --git a/akka-core/src/main/scala/remote/Cluster.scala b/akka-core/src/main/scala/remote/Cluster.scala index c2e9069a01..294ce5bd94 100644 --- a/akka-core/src/main/scala/remote/Cluster.scala +++ b/akka-core/src/main/scala/remote/Cluster.scala @@ -60,19 +60,18 @@ private[remote] object ClusterActor { abstract class BasicClusterActor extends ClusterActor { import ClusterActor._ - case class Message(sender : ADDR_T,msg : Array[Byte]) + case class Message(sender : ADDR_T, msg : Array[Byte]) case object PapersPlease extends ClusterMessage case class Papers(addresses: List[RemoteAddress]) extends ClusterMessage case object Block extends ClusterMessage case object Unblock extends ClusterMessage - case class View(othersPresent : Set[ADDR_T]) extends ClusterMessage + case class View(othersPresent: Set[ADDR_T]) extends ClusterMessage case class Zombie(address: ADDR_T) extends ClusterMessage case class RegisterLocalNode(server: RemoteAddress) extends ClusterMessage case class DeregisterLocalNode(server: RemoteAddress) extends ClusterMessage type ADDR_T - @volatile private var local: Node = Node(Nil) @volatile private var remotes: Map[ADDR_T, Node] = Map() @@ -206,17 +205,21 @@ abstract class BasicClusterActor extends ClusterActor { * Loads a specified ClusterActor and delegates to that instance. */ object Cluster extends Cluster with Logging { + lazy val DEFAULT_SERIALIZER_CLASS_NAME = Serializer.Java.getClass.getName + @volatile private[remote] var clusterActor: Option[ClusterActor] = None @volatile private[remote] var supervisor: Option[Supervisor] = None + + // FIXME Use the supervisor member field - private[remote] lazy val serializer: Serializer = { - val className = config.getString("akka.remote.cluster.serializer", Serializer.Java.getClass.getName) - Class.forName(className).newInstance.asInstanceOf[Serializer] - } + private[remote] lazy val serializer: Serializer = + Class.forName(config.getString("akka.remote.cluster.serializer", DEFAULT_SERIALIZER_CLASS_NAME)) + .newInstance.asInstanceOf[Serializer] - private[remote] def createClusterActor : Option[ClusterActor] = { + private[remote] def createClusterActor: Option[ClusterActor] = { val name = config.getString("akka.remote.cluster.actor") - + if (name.isEmpty) throw new IllegalArgumentException( + "Can't start cluster since the 'akka.remote.cluster.actor' configuration option is not defined") try { name map { fqn => val a = Class.forName(fqn).newInstance.asInstanceOf[ClusterActor] @@ -225,7 +228,7 @@ object Cluster extends Cluster with Logging { } } catch { - case e => log.error(e,"Couldn't load Cluster provider: [%s]",name.getOrElse("Not specified")); None + case e => log.error(e, "Couldn't load Cluster provider: [%s]", name.getOrElse("Not specified")); None } } @@ -250,10 +253,11 @@ object Cluster extends Cluster with Logging { def relayMessage(to: Class[_ <: Actor], msg: AnyRef): Unit = clusterActor.foreach(_.relayMessage(to, msg)) - def foreach(f : (RemoteAddress) => Unit) : Unit = clusterActor.foreach(_.foreach(f)) + def foreach(f: (RemoteAddress) => Unit) : Unit = clusterActor.foreach(_.foreach(f)) - def start : Unit = synchronized { - if(supervisor.isEmpty) { + def start: Unit = synchronized { + log.info("Starting up Cluster Service...") + if (supervisor.isEmpty) { for(actor <- createClusterActor; sup <- createSupervisor(actor)) { clusterActor = Some(actor) @@ -262,7 +266,8 @@ object Cluster extends Cluster with Logging { } } - def shutdown : Unit = synchronized { + def shutdown: Unit = synchronized { + log.info("Shutting down Cluster Service...") supervisor.foreach(_.stop) supervisor = None clusterActor = None diff --git a/akka-kernel/src/main/scala/Kernel.scala b/akka-kernel/src/main/scala/Kernel.scala index f63a50a0a7..7b2b2e4693 100644 --- a/akka-kernel/src/main/scala/Kernel.scala +++ b/akka-kernel/src/main/scala/Kernel.scala @@ -79,7 +79,7 @@ object Kernel extends Logging { (____ /__|_ \__|_ \(____ / \/ \/ \/ \/ """) - log.info(" Running version %s", Config.VERSION) + log.info(" Running version %s", Config.VERSION) log.info("==============================") } } diff --git a/akka-patterns/src/main/scala/Patterns.scala b/akka-patterns/src/main/scala/Patterns.scala index b967c07df7..9b7e55ccc9 100644 --- a/akka-patterns/src/main/scala/Patterns.scala +++ b/akka-patterns/src/main/scala/Patterns.scala @@ -3,14 +3,14 @@ package se.scalablesolutions.akka.actor.patterns import se.scalablesolutions.akka.actor.Actor object Patterns { - type PF[A,B] = PartialFunction[A,B] + type PF[A, B] = PartialFunction[A, B] /** * Creates a new PartialFunction whose isDefinedAt is a combination * of the two parameters, and whose apply is first to call filter.apply and then filtered.apply */ - def filter[A,B](filter : PF[A,Unit],filtered : PF[A,B]) : PF[A,B] = { - case a : A if filtered.isDefinedAt(a) && filter.isDefinedAt(a) => + def filter[A, B](filter: PF[A, Unit], filtered: PF[A, B]): PF[A, B] = { + case a: A if filtered.isDefinedAt(a) && filter.isDefinedAt(a) => filter(a) filtered(a) } @@ -18,39 +18,42 @@ object Patterns { /** * Interceptor is a filter(x,y) where x.isDefinedAt is considered to be always true */ - def intercept[A,B](interceptor : (A) => Unit, interceptee : PF[A,B]) : PF[A,B] = filter( - { case a if a.isInstanceOf[A] => interceptor(a) }, - interceptee + def intercept[A, B](interceptor: (A) => Unit, interceptee: PF[A, B]): PF[A, B] = filter( + {case a if a.isInstanceOf[A] => interceptor(a)}, + interceptee ) - + //FIXME 2.8, use default params with CyclicIterator - def loadBalancerActor(actors : => InfiniteIterator[Actor]) : Actor = new Actor with LoadBalancer { + def loadBalancerActor(actors: => InfiniteIterator[Actor]): Actor = new Actor with LoadBalancer { val seq = actors } - def dispatcherActor(routing : PF[Any,Actor], msgTransformer : (Any) => Any) : Actor = new Actor with Dispatcher { - override def transform(msg : Any) = msgTransformer(msg) + def dispatcherActor(routing: PF[Any, Actor], msgTransformer: (Any) => Any): Actor = new Actor with Dispatcher { + override def transform(msg: Any) = msgTransformer(msg) + def routes = routing } - - def dispatcherActor(routing : PF[Any,Actor]) : Actor = new Actor with Dispatcher { - def routes = routing + + def dispatcherActor(routing: PF[Any, Actor]): Actor = new Actor with Dispatcher { + def routes = routing } - def loggerActor(actorToLog : Actor, logger : (Any) => Unit) : Actor = dispatcherActor ( - { case _ => actorToLog }, + def loggerActor(actorToLog: Actor, logger: (Any) => Unit): Actor = dispatcherActor( + {case _ => actorToLog}, logger - ) + ) } -trait Dispatcher { self : Actor => +trait Dispatcher { + self: Actor => - protected def transform(msg : Any) : Any = msg - protected def routes : PartialFunction[Any,Actor] - - protected def dispatch : PartialFunction[Any,Unit] = { + protected def transform(msg: Any): Any = msg + + protected def routes: PartialFunction[Any, Actor] + + protected def dispatch: PartialFunction[Any, Unit] = { case a if routes.isDefinedAt(a) => { - if(self.sender.isDefined) + if (self.sender.isDefined) routes(a) forward transform(a) else routes(a) send transform(a) @@ -60,19 +63,22 @@ trait Dispatcher { self : Actor => def receive = dispatch } -trait LoadBalancer extends Dispatcher { self : Actor => - protected def seq : InfiniteIterator[Actor] +trait LoadBalancer extends Dispatcher { + self: Actor => + protected def seq: InfiniteIterator[Actor] - protected def routes = { case x if seq.hasNext => seq.next } + protected def routes = {case x if seq.hasNext => seq.next} } trait InfiniteIterator[T] extends Iterator[T] -class CyclicIterator[T](items : List[T]) extends InfiniteIterator[T] { - @volatile private[this] var current : List[T] = items +class CyclicIterator[T](items: List[T]) extends InfiniteIterator[T] { + @volatile private[this] var current: List[T] = items + def hasNext = items != Nil + def next = { - val nc = if(current == Nil) items else current + val nc = if (current == Nil) items else current current = nc.tail nc.head } diff --git a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala index b4ea8fc381..f52841b817 100644 --- a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala +++ b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala @@ -8,10 +8,11 @@ import se.scalablesolutions.akka.stm.TransactionManagement.transaction import se.scalablesolutions.akka.collection._ import se.scalablesolutions.akka.util.Logging -import org.codehaus.aspectwerkz.proxy.Uuid - +// FIXME move to 'stm' package + add message with more info class NoTransactionInScopeException extends RuntimeException +class StorageException(message: String) extends RuntimeException(message) + /** * Example Scala usage. *

diff --git a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala index 48945d6b8c..be214087f3 100644 --- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala +++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala @@ -72,11 +72,11 @@ private [akka] object RedisStorageBackend extends * base64(T1):base64("debasish.programming_language") -> "scala" * */ - def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]) { + def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]): Unit = withErrorHandling { insertMapStorageEntriesFor(name, List((key, value))) } - def insertMapStorageEntriesFor(name: String, entries: List[Tuple2[Array[Byte], Array[Byte]]]) { + def insertMapStorageEntriesFor(name: String, entries: List[Tuple2[Array[Byte], Array[Byte]]]): Unit = withErrorHandling { mset(entries.map(e => (makeRedisKey(name, e._1), new String(e._2)))) } @@ -89,22 +89,22 @@ private [akka] object RedisStorageBackend extends *

  • : is chosen since it cannot appear in base64 encoding charset
  • *
  • both parts of the key need to be based64 encoded since there can be spaces within each of them
  • */ - private [this] def makeRedisKey(name: String, key: Array[Byte]): String = { + private [this] def makeRedisKey(name: String, key: Array[Byte]): String = withErrorHandling { "%s:%s".format(new String(encode(name.getBytes)), new String(encode(key))) } - private [this] def makeKeyFromRedisKey(redisKey: String) = { + private [this] def makeKeyFromRedisKey(redisKey: String) = withErrorHandling { val nk = redisKey.split(':').map{e: String => decode(e.getBytes)} (nk(0), nk(1)) } - private [this] def mset(entries: List[(String, String)]) { + private [this] def mset(entries: List[(String, String)]): Unit = withErrorHandling { entries.foreach {e: (String, String) => db.set(e._1, e._2) } } - def removeMapStorageFor(name: String): Unit = { + def removeMapStorageFor(name: String): Unit = withErrorHandling { db.keys("%s:*".format(encode(name.getBytes))) match { case None => throw new Predef.NoSuchElementException(name + " not present") @@ -113,18 +113,19 @@ private [akka] object RedisStorageBackend extends } } - def removeMapStorageFor(name: String, key: Array[Byte]): Unit = { + def removeMapStorageFor(name: String, key: Array[Byte]): Unit = withErrorHandling { db.delete(makeRedisKey(name, key)) } - def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = + def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = withErrorHandling { db.get(makeRedisKey(name, key)) match { case None => throw new Predef.NoSuchElementException(new String(key) + " not present") case Some(s) => Some(s.getBytes) } + } - def getMapStorageSizeFor(name: String): Int = { + def getMapStorageSizeFor(name: String): Int = withErrorHandling { db.keys("%s:*".format(new String(encode(name.getBytes)))) match { case None => 0 case Some(keys) => @@ -132,7 +133,7 @@ private [akka] object RedisStorageBackend extends } } - def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = { + def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = withErrorHandling { db.keys("%s:*".format(new String(encode(name.getBytes)))) match { case None => throw new Predef.NoSuchElementException(name + " not present") @@ -143,7 +144,7 @@ private [akka] object RedisStorageBackend extends def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], - count: Int): List[(Array[Byte], Array[Byte])] = { + count: Int): List[(Array[Byte], Array[Byte])] = withErrorHandling { import scala.collection.immutable.TreeMap val wholeSorted = @@ -188,19 +189,19 @@ private [akka] object RedisStorageBackend extends } } - def insertVectorStorageEntryFor(name: String, element: Array[Byte]) { + def insertVectorStorageEntryFor(name: String, element: Array[Byte]): Unit = withErrorHandling { db.lpush(new String(encode(name.getBytes)), new String(element)) } - def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) { + def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]): Unit = withErrorHandling { elements.foreach(insertVectorStorageEntryFor(name, _)) } - def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) { + def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]): Unit = withErrorHandling { db.lset(new String(encode(name.getBytes)), index, new String(elem)) } - def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = { + def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = withErrorHandling { db.lindex(new String(encode(name.getBytes)), index) match { case None => throw new Predef.NoSuchElementException(name + " does not have element at " + index) @@ -208,7 +209,7 @@ private [akka] object RedisStorageBackend extends } } - def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = { + def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = withErrorHandling { /** * count is the max number of results to return. Start with * start or 0 (if start is not defined) and go until @@ -237,11 +238,11 @@ private [akka] object RedisStorageBackend extends } } - def insertRefStorageFor(name: String, element: Array[Byte]) { + def insertRefStorageFor(name: String, element: Array[Byte]): Unit = withErrorHandling { db.set(new String(encode(name.getBytes)), new String(element)) } - def getRefStorageFor(name: String): Option[Array[Byte]] = { + def getRefStorageFor(name: String): Option[Array[Byte]] = withErrorHandling { db.get(new String(encode(name.getBytes))) match { case None => throw new Predef.NoSuchElementException(name + " not present") @@ -250,12 +251,13 @@ private [akka] object RedisStorageBackend extends } // add to the end of the queue - def enqueue(name: String, item: Array[Byte]): Boolean = { + def enqueue(name: String, item: Array[Byte]): Boolean = withErrorHandling { db.rpush(new String(encode(name.getBytes)), new String(item)) } + // pop from the front of the queue - def dequeue(name: String): Option[Array[Byte]] = { + def dequeue(name: String): Option[Array[Byte]] = withErrorHandling { db.lpop(new String(encode(name.getBytes))) match { case None => throw new Predef.NoSuchElementException(name + " not present") @@ -265,7 +267,7 @@ private [akka] object RedisStorageBackend extends } // get the size of the queue - def size(name: String): Int = { + def size(name: String): Int = withErrorHandling { db.llen(new String(encode(name.getBytes))) match { case None => throw new Predef.NoSuchElementException(name + " not present") @@ -275,26 +277,28 @@ private [akka] object RedisStorageBackend extends // return an array of items currently stored in the queue // start is the item to begin, count is how many items to return - def peek(name: String, start: Int, count: Int): List[Array[Byte]] = count match { - case 1 => - db.lindex(new String(encode(name.getBytes)), start) match { - case None => - throw new Predef.NoSuchElementException("No element at " + start) - case Some(s) => - List(s.getBytes) - } - case n => - db.lrange(new String(encode(name.getBytes)), start, start + count - 1) match { - case None => - throw new Predef.NoSuchElementException( - "No element found between " + start + " and " + (start + count - 1)) - case Some(es) => - es.map(_.get.getBytes) - } + def peek(name: String, start: Int, count: Int): List[Array[Byte]] = withErrorHandling { + count match { + case 1 => + db.lindex(new String(encode(name.getBytes)), start) match { + case None => + throw new Predef.NoSuchElementException("No element at " + start) + case Some(s) => + List(s.getBytes) + } + case n => + db.lrange(new String(encode(name.getBytes)), start, start + count - 1) match { + case None => + throw new Predef.NoSuchElementException( + "No element found between " + start + " and " + (start + count - 1)) + case Some(es) => + es.map(_.get.getBytes) + } + } } // completely delete the queue - def remove(name: String): Boolean = { + def remove(name: String): Boolean = withErrorHandling { db.delete(new String(encode(name.getBytes))) match { case Some(1) => true case _ => false @@ -302,7 +306,7 @@ private [akka] object RedisStorageBackend extends } // add item to sorted set identified by name - def zadd(name: String, zscore: String, item: Array[Byte]): Boolean = { + def zadd(name: String, zscore: String, item: Array[Byte]): Boolean = withErrorHandling { db.zadd(new String(encode(name.getBytes)), zscore, new String(item)) match { case Some(1) => true case _ => false @@ -310,7 +314,7 @@ private [akka] object RedisStorageBackend extends } // remove item from sorted set identified by name - def zrem(name: String, item: Array[Byte]): Boolean = { + def zrem(name: String, item: Array[Byte]): Boolean = withErrorHandling { db.zrem(new String(encode(name.getBytes)), new String(item)) match { case Some(1) => true case _ => false @@ -318,7 +322,7 @@ private [akka] object RedisStorageBackend extends } // cardinality of the set identified by name - def zcard(name: String): Int = { + def zcard(name: String): Int = withErrorHandling { db.zcard(new String(encode(name.getBytes))) match { case None => throw new Predef.NoSuchElementException(name + " not present") @@ -326,7 +330,7 @@ private [akka] object RedisStorageBackend extends } } - def zscore(name: String, item: Array[Byte]): String = { + def zscore(name: String, item: Array[Byte]): String = withErrorHandling { db.zscore(new String(encode(name.getBytes)), new String(item)) match { case None => throw new Predef.NoSuchElementException(new String(item) + " not present") @@ -334,7 +338,7 @@ private [akka] object RedisStorageBackend extends } } - def zrange(name: String, start: Int, end: Int): List[Array[Byte]] = { + def zrange(name: String, start: Int, end: Int): List[Array[Byte]] = withErrorHandling { db.zrange(new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC, false) match { case None => throw new Predef.NoSuchElementException(name + " not present") @@ -343,5 +347,16 @@ private [akka] object RedisStorageBackend extends } } - def flushDB = db.flushDb + def flushDB = withErrorHandling(db.flushDb) + + private def withErrorHandling[T](body: => T): T = { + try { + body + } catch { + case e: java.lang.NullPointerException => + throw new StorageException("Could not connect to Redis server") + case e => + throw new StorageException("Error in Redis: " + e.getMessage) + } + } } diff --git a/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentActorSpec.scala b/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentActorSpec.scala index 86d4384b70..8c91f0ff61 100644 --- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentActorSpec.scala +++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisPersistentActorSpec.scala @@ -29,6 +29,7 @@ case object LogSize class AccountActor extends Transactor { private lazy val accountState = RedisStorage.newMap private lazy val txnLog = RedisStorage.newVector + //timeout = 5000 def receive = { // check balance @@ -86,6 +87,7 @@ class AccountActor extends Transactor { } @serializable class PersistentFailerActor extends Transactor { + //timeout = 5000 def receive = { case "Failure" => throw new RuntimeException("expected") @@ -138,7 +140,7 @@ class RedisPersistentActorSpec extends TestCase { bactor.start bactor !! Credit("a-123", 5000) - assertEquals(BigInt(5000), (bactor !! Balance("a-123")).get) + assertEquals(BigInt(5000), (bactor !! (Balance("a-123"), 5000)).get) val failer = new PersistentFailerActor failer.start @@ -147,7 +149,7 @@ class RedisPersistentActorSpec extends TestCase { fail("should throw exception") } catch { case e: RuntimeException => {}} - assertEquals(BigInt(5000), (bactor !! Balance("a-123")).get) + assertEquals(BigInt(5000), (bactor !! (Balance("a-123"), 5000)).get) // should not count the failed one assertEquals(3, (bactor !! LogSize).get) diff --git a/akka.iml b/akka.iml index a39c87020f..74542e8e48 100644 --- a/akka.iml +++ b/akka.iml @@ -15,6 +15,12 @@
    + + + + + +
    diff --git a/config/akka-reference.conf b/config/akka-reference.conf index 296b06428b..5a1c33497b 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -49,6 +49,7 @@ zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6 + service = on # FIXME add 'service = on' for name = "default" # The name of the cluster #actor = "se.scalablesolutions.akka.remote.JGroupsClusterActor" # FQN of an implementation of ClusterActor serializer = "se.scalablesolutions.akka.serialization.Serializer$Java" # FQN of the serializer class From 8091b6cb2634a04f0e1537d5e74dfec45e5ca833 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 4 Mar 2010 23:25:55 +0100 Subject: [PATCH 12/15] Fixing a bug in JGroupsClusterActor --- .../src/main/scala/JGroupsClusterActor.scala | 3 +- .../src/main/scala/ShoalClusterActor.scala | 9 ++-- akka-core/src/main/scala/remote/Cluster.scala | 47 +++++++++---------- .../src/main/scala/remote/RemoteServer.scala | 2 +- 4 files changed, 30 insertions(+), 31 deletions(-) diff --git a/akka-cluster/akka-cluster-jgroups/src/main/scala/JGroupsClusterActor.scala b/akka-cluster/akka-cluster-jgroups/src/main/scala/JGroupsClusterActor.scala index 2b6730dd60..12d93ef272 100644 --- a/akka-cluster/akka-cluster-jgroups/src/main/scala/JGroupsClusterActor.scala +++ b/akka-cluster/akka-cluster-jgroups/src/main/scala/JGroupsClusterActor.scala @@ -44,7 +44,8 @@ class JGroupsClusterActor extends BasicClusterActor { log debug "UNSUPPORTED: JGroupsClusterActor::unblock" //TODO HotSwap back and flush the buffer }) }) - channel.map(_.connect(name)) + + channel.foreach(_.connect(name)) } protected def toOneNode(dest : Address, msg: Array[Byte]): Unit = diff --git a/akka-cluster/akka-cluster-shoal/src/main/scala/ShoalClusterActor.scala b/akka-cluster/akka-cluster-shoal/src/main/scala/ShoalClusterActor.scala index c656bd4b81..3d83a46ef3 100644 --- a/akka-cluster/akka-cluster-shoal/src/main/scala/ShoalClusterActor.scala +++ b/akka-cluster/akka-cluster-shoal/src/main/scala/ShoalClusterActor.scala @@ -85,6 +85,7 @@ class ShoalClusterActor extends BasicClusterActor { */ protected def createCallback : CallBack = { import org.scala_tools.javautils.Imports._ + import ClusterActor._ val me = this new CallBack { def processNotification(signal : Signal) { @@ -92,10 +93,10 @@ class ShoalClusterActor extends BasicClusterActor { signal.acquire() if(isActive) { signal match { - case ms : MessageSignal => me send Message(ms.getMemberToken,ms.getMessage) - case jns : JoinNotificationSignal => me send View(Set[ADDR_T]() ++ jns.getCurrentCoreMembers.asScala - serverName) - case fss : FailureSuspectedSignal => me send Zombie(fss.getMemberToken) - case fns : FailureNotificationSignal => me send Zombie(fns.getMemberToken) + case ms : MessageSignal => me send Message[ADDR_T](ms.getMemberToken,ms.getMessage) + case jns : JoinNotificationSignal => me send View[ADDR_T](Set[ADDR_T]() ++ jns.getCurrentCoreMembers.asScala - serverName) + case fss : FailureSuspectedSignal => me send Zombie[ADDR_T](fss.getMemberToken) + case fns : FailureNotificationSignal => me send Zombie[ADDR_T](fns.getMemberToken) case _ => log.debug("Unhandled signal: [%s]",signal) } } diff --git a/akka-core/src/main/scala/remote/Cluster.scala b/akka-core/src/main/scala/remote/Cluster.scala index c2e9069a01..4313cfe98c 100644 --- a/akka-core/src/main/scala/remote/Cluster.scala +++ b/akka-core/src/main/scala/remote/Cluster.scala @@ -48,7 +48,15 @@ private[remote] object ClusterActor { sealed trait ClusterMessage private[remote] case class RelayedMessage(actorClassFQN: String, msg: AnyRef) extends ClusterMessage - + private[remote] case class Message[ADDR_T](sender : ADDR_T,msg : Array[Byte]) + private[remote] case object PapersPlease extends ClusterMessage + private[remote] case class Papers(addresses: List[RemoteAddress]) extends ClusterMessage + private[remote] case object Block extends ClusterMessage + private[remote] case object Unblock extends ClusterMessage + private[remote] case class View[ADDR_T](othersPresent : Set[ADDR_T]) extends ClusterMessage + private[remote] case class Zombie[ADDR_T](address: ADDR_T) extends ClusterMessage + private[remote] case class RegisterLocalNode(server: RemoteAddress) extends ClusterMessage + private[remote] case class DeregisterLocalNode(server: RemoteAddress) extends ClusterMessage private[remote] case class Node(endpoints: List[RemoteAddress]) } @@ -60,16 +68,6 @@ private[remote] object ClusterActor { abstract class BasicClusterActor extends ClusterActor { import ClusterActor._ - case class Message(sender : ADDR_T,msg : Array[Byte]) - case object PapersPlease extends ClusterMessage - case class Papers(addresses: List[RemoteAddress]) extends ClusterMessage - case object Block extends ClusterMessage - case object Unblock extends ClusterMessage - case class View(othersPresent : Set[ADDR_T]) extends ClusterMessage - case class Zombie(address: ADDR_T) extends ClusterMessage - case class RegisterLocalNode(server: RemoteAddress) extends ClusterMessage - case class DeregisterLocalNode(server: RemoteAddress) extends ClusterMessage - type ADDR_T @@ -85,14 +83,14 @@ abstract class BasicClusterActor extends ClusterActor { } def receive = { - case v @ View(members) => { + case v : View[ADDR_T] => { // Not present in the cluster anymore = presumably zombies // Nodes we have no prior knowledge existed = unknowns - val zombies = Set[ADDR_T]() ++ remotes.keySet -- members - val unknown = members -- remotes.keySet + val zombies = Set[ADDR_T]() ++ remotes.keySet -- v.othersPresent + val unknown = v.othersPresent -- remotes.keySet log debug ("Updating view") - log debug ("Other memebers: [%s]",members) + log debug ("Other memebers: [%s]",v.othersPresent) log debug ("Zombies: [%s]",zombies) log debug ("Unknowns: [%s]",unknown) @@ -101,10 +99,10 @@ abstract class BasicClusterActor extends ClusterActor { remotes = remotes -- zombies } - case Zombie(x) => { //Ask the presumed zombie for papers and prematurely treat it as dead - log debug ("Killing Zombie Node: %s", x) - broadcast(x :: Nil, PapersPlease) - remotes = remotes - x + case z : Zombie[ADDR_T] => { //Ask the presumed zombie for papers and prematurely treat it as dead + log debug ("Killing Zombie Node: %s", z.address) + broadcast(z.address :: Nil, PapersPlease) + remotes = remotes - z.address } case rm @ RelayedMessage(_, _) => { @@ -112,7 +110,8 @@ abstract class BasicClusterActor extends ClusterActor { broadcast(rm) } - case m @ Message(src,msg) => { + case m : Message[ADDR_T] => { + val (src,msg) = (m.sender,m.msg) (Cluster.serializer in (msg, None)) match { case PapersPlease => { @@ -207,7 +206,7 @@ abstract class BasicClusterActor extends ClusterActor { */ object Cluster extends Cluster with Logging { @volatile private[remote] var clusterActor: Option[ClusterActor] = None - @volatile private[remote] var supervisor: Option[Supervisor] = None + @volatile private[remote] var supervisor: Option[Supervisor] = None private[remote] lazy val serializer: Serializer = { val className = config.getString("akka.remote.cluster.serializer", Serializer.Java.getClass.getName) @@ -219,9 +218,7 @@ object Cluster extends Cluster with Logging { try { name map { fqn => - val a = Class.forName(fqn).newInstance.asInstanceOf[ClusterActor] - a.start - a + Class.forName(fqn).newInstance.asInstanceOf[ClusterActor] } } catch { @@ -235,7 +232,6 @@ object Cluster extends Cluster with Logging { RestartStrategy(OneForOne, 5, 1000, List(classOf[Exception])), Supervise(actor, LifeCycle(Permanent)) :: Nil) ).newInstance - sup.start Some(sup) } @@ -258,6 +254,7 @@ object Cluster extends Cluster with Logging { sup <- createSupervisor(actor)) { clusterActor = Some(actor) supervisor = Some(sup) + sup.start } } } diff --git a/akka-core/src/main/scala/remote/RemoteServer.scala b/akka-core/src/main/scala/remote/RemoteServer.scala index 6da2ceea99..02cf98bcd2 100644 --- a/akka-core/src/main/scala/remote/RemoteServer.scala +++ b/akka-core/src/main/scala/remote/RemoteServer.scala @@ -58,7 +58,7 @@ object RemoteNode extends RemoteServer */ object RemoteServer { val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost") - val PORT = config.getInt("akka.remote.server.port", 9999) + val PORT = config.getInt("akka.remote.server.port", 9966) val CONNECTION_TIMEOUT_MILLIS = config.getInt("akka.remote.server.connection-timeout", 1000) From efa0cc0d5abb0951404b940c7549a5314141ab39 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Fri, 5 Mar 2010 16:07:06 +0100 Subject: [PATCH 13/15] Fixed last persistence issues with new STM, all test pass --- .../scala/CassandraPersistentActorSpec.scala | 16 +++++----------- .../test/scala/MongoPersistentActorSpec.scala | 13 ++++++------- 2 files changed, 11 insertions(+), 18 deletions(-) diff --git a/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraPersistentActorSpec.scala b/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraPersistentActorSpec.scala index 0e232f5ce9..a7fed923eb 100644 --- a/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraPersistentActorSpec.scala +++ b/akka-persistence/akka-persistence-cassandra/src/test/scala/CassandraPersistentActorSpec.scala @@ -1,13 +1,9 @@ package se.scalablesolutions.akka.state -import se.scalablesolutions.akka.actor.Actor - -import junit.framework.TestCase +import se.scalablesolutions.akka.actor.{Actor, Transactor} import org.junit.Test import org.junit.Assert._ -import org.apache.cassandra.service.CassandraDaemon -import org.junit.BeforeClass import org.junit.Before import org.scalatest.junit.JUnitSuite @@ -28,9 +24,8 @@ case class SetRefStateOneWay(key: String) case class SuccessOneWay(key: String, value: String) case class FailureOneWay(key: String, value: String, failer: Actor) -class CassandraPersistentActor extends Actor { +class CassandraPersistentActor extends Transactor { timeout = 100000 - makeTransactionRequired private lazy val mapState = CassandraStorage.newMap private lazy val vectorState = CassandraStorage.newVector @@ -66,8 +61,7 @@ class CassandraPersistentActor extends Actor { } } -@serializable class PersistentFailerActor extends Actor { - makeTransactionRequired +@serializable class PersistentFailerActor extends Transactor { def receive = { case "Failure" => throw new RuntimeException("expected") @@ -76,8 +70,8 @@ class CassandraPersistentActor extends Actor { class CassandraPersistentActorSpec extends JUnitSuite { - @Before - def startCassandra = EmbeddedCassandraService.start + //@Before + //def startCassandra = EmbeddedCassandraService.start @Test def testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess = { diff --git a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala index 8681ebadb9..8ad5d94355 100644 --- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala +++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala @@ -8,7 +8,7 @@ import org.junit.Assert._ import _root_.dispatch.json.{JsNumber, JsValue} import _root_.dispatch.json.Js._ -import se.scalablesolutions.akka.actor.Actor +import se.scalablesolutions.akka.actor.{Transactor, Actor} /** * A persistent actor based on MongoDB storage. @@ -29,10 +29,10 @@ case class MultiDebit(accountNo: String, amounts: List[BigInt], failer: Actor) case class Credit(accountNo: String, amount: BigInt) case object LogSize -class BankAccountActor extends Actor { - makeTransactionRequired - private val accountState = MongoStorage.newMap - private val txnLog = MongoStorage.newVector +class BankAccountActor extends Transactor { + + private lazy val accountState = MongoStorage.newMap + private lazy val txnLog = MongoStorage.newVector def receive: PartialFunction[Any, Unit] = { // check balance @@ -91,8 +91,7 @@ class BankAccountActor extends Actor { } } -@serializable class PersistentFailerActor extends Actor { - makeTransactionRequired +@serializable class PersistentFailerActor extends Transactor { def receive = { case "Failure" => throw new RuntimeException("expected") From 57009b112bb43b2157a7159b7612fc97142b67fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Fri, 5 Mar 2010 17:44:11 +0100 Subject: [PATCH 14/15] removed log.trace that gave bad perf --- akka-core/src/main/scala/actor/Actor.scala | 2 +- akka-core/src/test/scala/PerformanceTest.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-core/src/main/scala/actor/Actor.scala b/akka-core/src/main/scala/actor/Actor.scala index a6b46d903d..60f3967e6c 100644 --- a/akka-core/src/main/scala/actor/Actor.scala +++ b/akka-core/src/main/scala/actor/Actor.scala @@ -873,7 +873,7 @@ trait Actor extends TransactionManagement { * Callback for the dispatcher. E.g. single entry point to the user code and all protected[this] methods. */ private[akka] def invoke(messageHandle: MessageInvocation) = synchronized { - log.trace("%s is invoked with message %s", toString, messageHandle) + //log.trace("%s is invoked with message %s", toString, messageHandle) try { if (TransactionManagement.isTransactionalityEnabled) transactionalDispatch(messageHandle) else dispatch(messageHandle) diff --git a/akka-core/src/test/scala/PerformanceTest.scala b/akka-core/src/test/scala/PerformanceTest.scala index d58d075202..778e4d45cd 100644 --- a/akka-core/src/test/scala/PerformanceTest.scala +++ b/akka-core/src/test/scala/PerformanceTest.scala @@ -1,4 +1,4 @@ -package test +package se.scalablesolutions.akka import org.scalatest.junit.JUnitSuite import org.junit.Test @@ -279,7 +279,7 @@ class PerformanceTest extends JUnitSuite { var nrOfMessages = 2000000 var nrOfActors = 4 - var akkaTime = stressTestAkkaActors(nrOfMessages, nrOfActors, 1000 * 20) + var akkaTime = stressTestAkkaActors(nrOfMessages, nrOfActors, 1000 * 30) var scalaTime = stressTestScalaActors(nrOfMessages, nrOfActors, 1000 * 40) var ratio: Double = scalaTime.toDouble / akkaTime.toDouble From 02fe5ae2adba55f41944dd579f58cc7f5b55dd37 Mon Sep 17 00:00:00 2001 From: Martin Krasser Date: Fri, 5 Mar 2010 21:38:23 +0100 Subject: [PATCH 15/15] do not include *QSpec.java for testing --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 480ad8723a..49023e42c9 100644 --- a/pom.xml +++ b/pom.xml @@ -306,7 +306,7 @@ **/*Test.java - **/*Spec.java +