Separated core from modules, moved modules to akka-modules repository

This commit is contained in:
Jonas Bonér 2010-11-23 15:31:28 +01:00
parent 69777ca20b
commit bd29ede999
306 changed files with 5 additions and 25602 deletions

0
README
View file

View file

@ -1,267 +0,0 @@
package akka.amqp;
import org.multiverse.api.latches.StandardLatch;
import scala.Option;
import akka.actor.ActorRef;
import akka.actor.ActorRegistry;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
import akka.amqp.rpc.RPC;
import akka.remote.protocol.RemoteProtocol;
import akka.japi.Function;
import akka.japi.Procedure;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@SuppressWarnings({"unchecked"})
public class ExampleSessionJava {
public static void main(String... args) {
new ExampleSessionJava();
}
public ExampleSessionJava() {
printTopic("DIRECT");
direct();
printTopic("CALLBACK");
callback();
printTopic("EASY STRING PRODUCER AND CONSUMER");
easyStringProducerConsumer();
printTopic("EASY PROTOBUF PRODUCER AND CONSUMER");
easyProtobufProducerConsumer();
printTopic("EASY STRING RPC");
easyStringRpc();
printTopic("EASY PROTOBUF RPC");
easyProtobufRpc();
// postStop everything the amqp tree except the main AMQP supervisor
// all connections/consumers/producers will be stopped
AMQP.shutdownAll();
ActorRegistry.shutdownAll();
printTopic("Happy hAkking :-)");
System.exit(0);
}
private void printTopic(String topic) {
System.out.println("");
System.out.println("==== " + topic + " ===");
System.out.println("");
try {
TimeUnit.SECONDS.sleep(2);
} catch (InterruptedException ignore) {
}
}
private void direct() {
// defaults to amqp://guest:guest@localhost:5672/
ActorRef connection = AMQP.newConnection();
AMQP.ExchangeParameters exchangeParameters = new AMQP.ExchangeParameters("my_direct_exchange", Direct.getInstance());
ActorRef deliveryHandler = UntypedActor.actorOf(DirectDeliveryHandlerActor.class);
AMQP.ConsumerParameters consumerParameters = new AMQP.ConsumerParameters("some.routing", deliveryHandler, exchangeParameters);
ActorRef consumer = AMQP.newConsumer(connection, consumerParameters);
ActorRef producer = AMQP.newProducer(connection, new AMQP.ProducerParameters(exchangeParameters));
producer.sendOneWay(new Message("@jonas_boner: You sucked!!".getBytes(), "some.routing"));
}
private void callback() {
final CountDownLatch channelCountdown = new CountDownLatch(2);
ActorRef connectionCallback = UntypedActor.actorOf(ConnectionCallbackActor.class);
connectionCallback.start();
AMQP.ConnectionParameters connectionParameters = new AMQP.ConnectionParameters(connectionCallback);
ActorRef connection = AMQP.newConnection(connectionParameters);
ActorRef channelCallback = UntypedActor.actorOf(new UntypedActorFactory() {
public UntypedActor create() {
return new ChannelCallbackActor(channelCountdown);
}
});
channelCallback.start();
AMQP.ExchangeParameters exchangeParameters = new AMQP.ExchangeParameters("my_callback_exchange", Direct.getInstance());
AMQP.ChannelParameters channelParameters = new AMQP.ChannelParameters(channelCallback);
ActorRef dummyHandler = UntypedActor.actorOf(DummyActor.class);
AMQP.ConsumerParameters consumerParameters = new AMQP.ConsumerParameters("callback.routing", dummyHandler, exchangeParameters, channelParameters);
ActorRef consumer = AMQP.newConsumer(connection, consumerParameters);
ActorRef producer = AMQP.newProducer(connection, new AMQP.ProducerParameters(exchangeParameters, channelParameters));
// Wait until both channels (producer & consumer) are started before stopping the connection
try {
channelCountdown.await(2, TimeUnit.SECONDS);
} catch (InterruptedException ignore) {
}
connection.stop();
}
public void easyStringProducerConsumer() {
ActorRef connection = AMQP.newConnection();
String exchangeName = "easy.string";
// listen by default to:
// exchange = optional exchangeName
// routingKey = provided routingKey or <exchangeName>.request
// queueName = <routingKey>.in
Procedure<String> procedure = new Procedure<String>() {
public void apply(String message) {
System.out.println("### >> Received message: " + message);
}
};
AMQP.newStringConsumer(connection, procedure, exchangeName);
// send by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
AMQP.ProducerClient<String> producer = AMQP.newStringProducer(connection, exchangeName);
producer.send("This shit is easy!");
}
public void easyProtobufProducerConsumer() {
ActorRef connection = AMQP.newConnection();
String exchangeName = "easy.protobuf";
Procedure<RemoteProtocol.AddressProtocol> procedure = new Procedure<RemoteProtocol.AddressProtocol>() {
public void apply(RemoteProtocol.AddressProtocol message) {
System.out.println("### >> Received message: " + message);
}
};
AMQP.newProtobufConsumer(connection, procedure, exchangeName, RemoteProtocol.AddressProtocol.class);
AMQP.ProducerClient<RemoteProtocol.AddressProtocol> producerClient = AMQP.newProtobufProducer(connection, exchangeName);
producerClient.send(RemoteProtocol.AddressProtocol.newBuilder().setHostname("akkarocks.com").setPort(1234).build());
}
public void easyStringRpc() {
ActorRef connection = AMQP.newConnection();
String exchangeName = "easy.stringrpc";
// listen by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
// queueName = <routingKey>.in
RPC.newStringRpcServer(connection, exchangeName, new Function<String, String>() {
public String apply(String request) {
System.out.println("### >> Got request: " + request);
return "Response to: '" + request + "'";
}
});
// send by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
RPC.RpcClient<String, String> stringRpcClient = RPC.newStringRpcClient(connection, exchangeName);
Option<String> response = stringRpcClient.call("AMQP Rocks!");
System.out.println("### >> Got response: " + response);
final StandardLatch standardLatch = new StandardLatch();
stringRpcClient.callAsync("AMQP is dead easy", new Procedure<String>() {
public void apply(String request) {
System.out.println("### >> This is handled async: " + request);
standardLatch.open();
}
});
try {
standardLatch.tryAwait(2, TimeUnit.SECONDS);
} catch (InterruptedException ignore) {
}
}
public void easyProtobufRpc() {
ActorRef connection = AMQP.newConnection();
String exchangeName = "easy.protobuf.rpc";
RPC.newProtobufRpcServer(connection, exchangeName, new Function<RemoteProtocol.AddressProtocol, RemoteProtocol.AddressProtocol>() {
public RemoteProtocol.AddressProtocol apply(RemoteProtocol.AddressProtocol request) {
return RemoteProtocol.AddressProtocol.newBuilder().setHostname(request.getHostname()).setPort(request.getPort()).build();
}
}, RemoteProtocol.AddressProtocol.class);
RPC.RpcClient<RemoteProtocol.AddressProtocol, RemoteProtocol.AddressProtocol> protobufRpcClient =
RPC.newProtobufRpcClient(connection, exchangeName, RemoteProtocol.AddressProtocol.class);
scala.Option<RemoteProtocol.AddressProtocol> response =
protobufRpcClient.call(RemoteProtocol.AddressProtocol.newBuilder().setHostname("localhost").setPort(4321).build());
System.out.println("### >> Got response: " + response);
}
}
class DummyActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
// not used
}
}
class ChannelCallbackActor extends UntypedActor {
private final CountDownLatch channelCountdown;
public ChannelCallbackActor(CountDownLatch channelCountdown) {
this.channelCountdown = channelCountdown;
}
public void onReceive(Object message) throws Exception {
if (Started.getInstance().getClass().isAssignableFrom(message.getClass())) {
System.out.println("### >> Channel callback: Started");
channelCountdown.countDown();
} else if (Restarting.getInstance().getClass().isAssignableFrom(message.getClass())) {
} else if (Stopped.getInstance().getClass().isAssignableFrom(message.getClass())) {
System.out.println("### >> Channel callback: Stopped");
} else throw new IllegalArgumentException("Unknown message: " + message);
}
}
class ConnectionCallbackActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
if (Connected.getInstance().getClass().isAssignableFrom(message.getClass())) {
System.out.println("### >> Connection callback: Connected!");
} else if (Reconnecting.getInstance().getClass().isAssignableFrom(message.getClass())) {
} else if (Disconnected.getInstance().getClass().isAssignableFrom(message.getClass())) {
System.out.println("### >> Connection callback: Disconnected!");
} else throw new IllegalArgumentException("Unknown message: " + message);
}
}
class DirectDeliveryHandlerActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
if (Delivery.class.isAssignableFrom(message.getClass())) {
Delivery delivery = (Delivery) message;
System.out.println("### >> @george_bush received message from: " + new String(delivery.payload()));
} else throw new IllegalArgumentException("Unknown message: " + message);
}
}

View file

@ -1,489 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
import akka.actor.{Actor, ActorRef}
import akka.actor.Actor._
import akka.config.Supervision.OneForOneStrategy
import com.rabbitmq.client.{ReturnListener, ShutdownListener, ConnectionFactory}
import ConnectionFactory._
import com.rabbitmq.client.AMQP.BasicProperties
import java.lang.{String, IllegalArgumentException}
import reflect.Manifest
import akka.japi.Procedure
import akka.dispatch.Dispatchers
/**
* AMQP Actor API. Implements Connection, Producer and Consumer materialized as Actors.
*
* @see akka.amqp.ExampleSession
*
* @author Irmo Manie
*/
object AMQP {
lazy val consumerDispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("amqp-consumers").build
lazy val producerDispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("amqp-producers").build
/**
* Parameters used to make the connection to the amqp broker. Uses the rabbitmq defaults.
*/
case class ConnectionParameters(
host: String = DEFAULT_HOST,
port: Int = DEFAULT_AMQP_PORT,
username: String = DEFAULT_USER,
password: String = DEFAULT_PASS,
virtualHost: String = DEFAULT_VHOST,
initReconnectDelay: Long = 5000,
connectionCallback: Option[ActorRef] = None) {
// Needed for Java API usage
def this() = this (DEFAULT_HOST, DEFAULT_AMQP_PORT, DEFAULT_USER, DEFAULT_PASS, DEFAULT_VHOST, 5000, None)
// Needed for Java API usage
def this(host: String, port: Int, username: String, password: String, virtualHost: String) =
this (host, port, username, password, virtualHost, 5000, None)
// Needed for Java API usage
def this(host: String, port: Int, username: String, password: String, virtualHost: String, initReconnectDelay: Long, connectionCallback: ActorRef) =
this (host, port, username, password, virtualHost, initReconnectDelay, Some(connectionCallback))
// Needed for Java API usage
def this(connectionCallback: ActorRef) =
this (DEFAULT_HOST, DEFAULT_AMQP_PORT, DEFAULT_USER, DEFAULT_PASS, DEFAULT_VHOST, 5000, Some(connectionCallback))
}
/**
* Additional parameters for the channel
*/
case class ChannelParameters(
shutdownListener: Option[ShutdownListener] = None,
channelCallback: Option[ActorRef] = None,
prefetchSize: Int = 0) {
// Needed for Java API usage
def this() = this (None, None)
// Needed for Java API usage
def this(channelCallback: ActorRef) = this (None, Some(channelCallback))
// Needed for Java API usage
def this(shutdownListener: ShutdownListener, channelCallback: ActorRef) =
this (Some(shutdownListener), Some(channelCallback))
}
/**
* Declaration type used for either exchange or queue declaration
*/
sealed trait Declaration
case object NoActionDeclaration extends Declaration {
def getInstance() = this // Needed for Java API usage
}
case object PassiveDeclaration extends Declaration {
def getInstance() = this // Needed for Java API usage
}
case class ActiveDeclaration(durable: Boolean = false, autoDelete: Boolean = true, exclusive: Boolean = false) extends Declaration {
// Needed for Java API usage
def this() = this (false, true, false)
// Needed for Java API usage
def this(durable: Boolean, autoDelete: Boolean) = this (durable, autoDelete, false)
}
/**
* Exchange specific parameters
*/
case class ExchangeParameters(
exchangeName: String,
exchangeType: ExchangeType = Topic,
exchangeDeclaration: Declaration = ActiveDeclaration(),
configurationArguments: Map[String, AnyRef] = Map.empty) {
// Needed for Java API usage
def this(exchangeName: String) =
this (exchangeName, Topic, ActiveDeclaration(), Map.empty)
// Needed for Java API usage
def this(exchangeName: String, exchangeType: ExchangeType) =
this (exchangeName, exchangeType, ActiveDeclaration(), Map.empty)
// Needed for Java API usage
def this(exchangeName: String, exchangeType: ExchangeType, exchangeDeclaration: Declaration) =
this (exchangeName, exchangeType, exchangeDeclaration, Map.empty)
}
/**
* Producer specific parameters
*/
case class ProducerParameters(
exchangeParameters: Option[ExchangeParameters] = None,
producerId: Option[String] = None,
returnListener: Option[ReturnListener] = None,
channelParameters: Option[ChannelParameters] = None) {
def this() = this (None, None, None, None)
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters) = this (Some(exchangeParameters), None, None, None)
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters, producerId: String) =
this (Some(exchangeParameters), Some(producerId), None, None)
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters, returnListener: ReturnListener) =
this (Some(exchangeParameters), None, Some(returnListener), None)
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters, channelParameters: ChannelParameters) =
this (Some(exchangeParameters), None, None, Some(channelParameters))
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters, producerId: String, returnListener: ReturnListener, channelParameters: ChannelParameters) =
this (Some(exchangeParameters), Some(producerId), Some(returnListener), Some(channelParameters))
}
/**
* Consumer specific parameters
*/
case class ConsumerParameters(
routingKey: String,
deliveryHandler: ActorRef,
queueName: Option[String] = None,
exchangeParameters: Option[ExchangeParameters] = None,
queueDeclaration: Declaration = ActiveDeclaration(),
selfAcknowledging: Boolean = true,
channelParameters: Option[ChannelParameters] = None) {
if (queueName.isEmpty) {
queueDeclaration match {
case ActiveDeclaration(true, _, _) =>
throw new IllegalArgumentException("A queue name is required when requesting a durable queue.")
case PassiveDeclaration =>
throw new IllegalArgumentException("A queue name is required when requesting passive declaration.")
case _ => () // ignore
}
}
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef) =
this (routingKey, deliveryHandler, None, None, ActiveDeclaration(), true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, None, None, ActiveDeclaration(), true, Some(channelParameters))
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, selfAcknowledging: Boolean) =
this (routingKey, deliveryHandler, None, None, ActiveDeclaration(), selfAcknowledging, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, selfAcknowledging: Boolean, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, None, None, ActiveDeclaration(), selfAcknowledging, Some(channelParameters))
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String) =
this (routingKey, deliveryHandler, Some(queueName), None, ActiveDeclaration(), true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, queueDeclaration: Declaration, selfAcknowledging: Boolean, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, Some(queueName), None, queueDeclaration, selfAcknowledging, Some(channelParameters))
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, exchangeParameters: ExchangeParameters) =
this (routingKey, deliveryHandler, None, Some(exchangeParameters), ActiveDeclaration(), true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, exchangeParameters: ExchangeParameters, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, None, Some(exchangeParameters), ActiveDeclaration(), true, Some(channelParameters))
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, exchangeParameters: ExchangeParameters, selfAcknowledging: Boolean) =
this (routingKey, deliveryHandler, None, Some(exchangeParameters), ActiveDeclaration(), selfAcknowledging, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, exchangeParameters: ExchangeParameters) =
this (routingKey, deliveryHandler, Some(queueName), Some(exchangeParameters), ActiveDeclaration(), true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, exchangeParameters: ExchangeParameters, queueDeclaration: Declaration) =
this (routingKey, deliveryHandler, Some(queueName), Some(exchangeParameters), queueDeclaration, true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, exchangeParameters: ExchangeParameters, queueDeclaration: Declaration, selfAcknowledging: Boolean) =
this (routingKey, deliveryHandler, Some(queueName), Some(exchangeParameters), queueDeclaration, selfAcknowledging, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, exchangeParameters: ExchangeParameters, queueDeclaration: Declaration, selfAcknowledging: Boolean, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, Some(queueName), Some(exchangeParameters), queueDeclaration, selfAcknowledging, Some(channelParameters))
// How about that for some overloading... huh? :P (yes, I know, there are still possibilities left...sue me!)
// Who said java is easy :(
}
def newConnection(connectionParameters: ConnectionParameters = new ConnectionParameters()): ActorRef = {
val connection = actorOf(new FaultTolerantConnectionActor(connectionParameters))
supervisor.startLink(connection)
connection ! Connect
connection
}
// Needed for Java API usage
def newConnection(): ActorRef = {
newConnection(new ConnectionParameters())
}
def newProducer(connection: ActorRef, producerParameters: ProducerParameters): ActorRef = {
val producer: ActorRef = Actor.actorOf(new ProducerActor(producerParameters))
producer.dispatcher = producerDispatcher
connection.startLink(producer)
producer ! Start
producer
}
def newConsumer(connection: ActorRef, consumerParameters: ConsumerParameters): ActorRef = {
val consumer: ActorRef = actorOf(new ConsumerActor(consumerParameters))
consumer.dispatcher = consumerDispatcher
val handler = consumerParameters.deliveryHandler
if (handler.isUnstarted) handler.dispatcher = consumerDispatcher
if (handler.supervisor.isEmpty) consumer.startLink(handler)
connection.startLink(consumer)
consumer ! Start
consumer
}
/**
* Convenience
*/
class ProducerClient[O](client: ActorRef, routingKey: String, toBinary: ToBinary[O]) {
// Needed for Java API usage
def send(request: O): Unit = {
send(request, None)
}
// Needed for Java API usage
def send(request: O, replyTo: String): Unit = {
send(request, Some(replyTo))
}
def send(request: O, replyTo: Option[String] = None) = {
val basicProperties = new BasicProperties
basicProperties.setReplyTo(replyTo.getOrElse(null))
client ! Message(toBinary.toBinary(request), routingKey, false, false, Some(basicProperties))
}
def stop() = client.stop
}
// Needed for Java API usage
def newStringProducer(connection: ActorRef,
exchangeName: String): ProducerClient[String] = {
newStringProducer(connection, Some(exchangeName))
}
// Needed for Java API usage
def newStringProducer(connection: ActorRef,
exchangeName: String,
routingKey: String): ProducerClient[String] = {
newStringProducer(connection, Some(exchangeName), Some(routingKey))
}
// Needed for Java API usage
def newStringProducer(connection: ActorRef,
exchangeName: String,
routingKey: String,
producerId: String): ProducerClient[String] = {
newStringProducer(connection, Some(exchangeName), Some(routingKey), Some(producerId))
}
def newStringProducer(connection: ActorRef,
exchangeName: Option[String],
routingKey: Option[String] = None,
producerId: Option[String] = None): ProducerClient[String] = {
if (exchangeName.isEmpty && routingKey.isEmpty) {
throw new IllegalArgumentException("Either exchange name or routing key is mandatory")
}
val exchangeParameters = exchangeName.flatMap(name => Some(ExchangeParameters(name)))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName.get))
val producerRef = newProducer(connection, ProducerParameters(exchangeParameters, producerId))
val toBinary = new ToBinary[String] {
def toBinary(t: String) = t.getBytes
}
new ProducerClient(producerRef, rKey, toBinary)
}
// Needed for Java API usage
def newStringConsumer(connection: ActorRef,
handler: Procedure[String],
exchangeName: String): ActorRef = {
newStringConsumer(connection, handler.apply _, Some(exchangeName))
}
// Needed for Java API usage
def newStringConsumer(connection: ActorRef,
handler: Procedure[String],
exchangeName: String,
routingKey: String): ActorRef = {
newStringConsumer(connection, handler.apply _, Some(exchangeName), Some(routingKey))
}
// Needed for Java API usage
def newStringConsumer(connection: ActorRef,
handler: Procedure[String],
exchangeName: String,
routingKey: String,
queueName: String): ActorRef = {
newStringConsumer(connection, handler.apply _, Some(exchangeName), Some(routingKey), Some(queueName))
}
def newStringConsumer(connection: ActorRef,
handler: String => Unit,
exchangeName: Option[String],
routingKey: Option[String] = None,
queueName: Option[String] = None): ActorRef = {
if (exchangeName.isEmpty && routingKey.isEmpty) {
throw new IllegalArgumentException("Either exchange name or routing key is mandatory")
}
val deliveryHandler = actorOf( new Actor {
def receive = { case Delivery(payload, _, _, _, _, _) => handler.apply(new String(payload)) }
} ).start
val exchangeParameters = exchangeName.flatMap(name => Some(ExchangeParameters(name)))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName.get))
val qName = queueName.getOrElse("%s.in".format(rKey))
newConsumer(connection, ConsumerParameters(rKey, deliveryHandler, Some(qName), exchangeParameters))
}
// Needed for Java API usage
def newProtobufProducer[O <: com.google.protobuf.Message](connection: ActorRef,
exchangeName: String): ProducerClient[O] = {
newProtobufProducer(connection, Some(exchangeName))
}
// Needed for Java API usage
def newProtobufProducer[O <: com.google.protobuf.Message](connection: ActorRef,
exchangeName: String,
routingKey: String): ProducerClient[O] = {
newProtobufProducer(connection, Some(exchangeName), Some(routingKey))
}
// Needed for Java API usage
def newProtobufProducer[O <: com.google.protobuf.Message](connection: ActorRef,
exchangeName: String,
routingKey: String,
producerId: String): ProducerClient[O] = {
newProtobufProducer(connection, Some(exchangeName), Some(routingKey), Some(producerId))
}
def newProtobufProducer[O <: com.google.protobuf.Message](connection: ActorRef,
exchangeName: Option[String],
routingKey: Option[String] = None,
producerId: Option[String] = None): ProducerClient[O] = {
if (exchangeName.isEmpty && routingKey.isEmpty) {
throw new IllegalArgumentException("Either exchange name or routing key is mandatory")
}
val exchangeParameters = exchangeName.flatMap(name => Some(ExchangeParameters(name)))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName.get))
val producerRef = newProducer(connection, ProducerParameters(exchangeParameters, producerId))
new ProducerClient(producerRef, rKey, new ToBinary[O] {
def toBinary(t: O) = t.toByteArray
})
}
// Needed for Java API usage
def newProtobufConsumer[I <: com.google.protobuf.Message](connection: ActorRef,
handler: Procedure[I],
exchangeName: String,
clazz: Class[I]): ActorRef = {
implicit val manifest = Manifest.classType[I](clazz)
newProtobufConsumer[I](connection, handler.apply _, Some(exchangeName))
}
// Needed for Java API usage
def newProtobufConsumer[I <: com.google.protobuf.Message](connection: ActorRef,
handler: Procedure[I],
exchangeName: String,
routingKey: String,
clazz: Class[I]): ActorRef = {
implicit val manifest = Manifest.classType[I](clazz)
newProtobufConsumer[I](connection, handler.apply _, Some(exchangeName), Some(routingKey))
}
// Needed for Java API usage
def newProtobufConsumer[I <: com.google.protobuf.Message](connection: ActorRef,
handler: Procedure[I],
exchangeName: String,
routingKey: String,
queueName: String,
clazz: Class[I]): ActorRef = {
implicit val manifest = Manifest.classType[I](clazz)
newProtobufConsumer[I](connection, handler.apply _, Some(exchangeName), Some(routingKey), Some(queueName))
}
def newProtobufConsumer[I <: com.google.protobuf.Message](connection: ActorRef,
handler: I => Unit,
exchangeName: Option[String],
routingKey: Option[String] = None,
queueName: Option[String] = None)(implicit manifest: Manifest[I]): ActorRef = {
if (exchangeName.isEmpty && routingKey.isEmpty) {
throw new IllegalArgumentException("Either exchange name or routing key is mandatory")
}
val deliveryHandler = actorOf(new Actor {
def receive = { case Delivery(payload, _, _, _, _, _) => handler.apply(createProtobufFromBytes[I](payload)) }
}).start
val exchangeParameters = exchangeName.flatMap(name => Some(ExchangeParameters(name)))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName.get))
val qName = queueName.getOrElse("%s.in".format(rKey))
newConsumer(connection, ConsumerParameters(rKey, deliveryHandler, Some(qName), exchangeParameters))
}
/**
* Main supervisor
*/
class AMQPSupervisorActor extends Actor {
import self._
faultHandler = OneForOneStrategy(List(classOf[Throwable]))
def receive = {
case _ => {} // ignore all messages
}
}
private val supervisor = actorOf(new AMQPSupervisorActor).start
def shutdownAll() = {
supervisor.shutdownLinkedActors
}
/**
* Serialization stuff
*/
trait FromBinary[T] {
def fromBinary(bytes: Array[Byte]): T
}
trait ToBinary[T] {
def toBinary(t: T): Array[Byte]
}
private val ARRAY_OF_BYTE_ARRAY = Array[Class[_]](classOf[Array[Byte]])
private[amqp] def createProtobufFromBytes[I <: com.google.protobuf.Message](bytes: Array[Byte])(implicit manifest: Manifest[I]): I = {
manifest.erasure.getDeclaredMethod("parseFrom", ARRAY_OF_BYTE_ARRAY: _*).invoke(null, bytes).asInstanceOf[I]
}
}

View file

@ -1,92 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
import akka.actor.ActorRef
import com.rabbitmq.client.AMQP.BasicProperties
import com.rabbitmq.client.ShutdownSignalException
sealed trait AMQPMessage
sealed trait InternalAMQPMessage extends AMQPMessage
case class Message(
payload: Array[Byte],
routingKey: String,
mandatory: Boolean = false,
immediate: Boolean = false,
properties: Option[BasicProperties] = None) extends AMQPMessage {
// Needed for Java API usage
def this(payload: Array[Byte], routingKey: String) = this(payload, routingKey, false, false, None)
// Needed for Java API usage
def this(payload: Array[Byte], routingKey: String, mandatory: Boolean, immediate: Boolean) =
this(payload, routingKey, mandatory, immediate, None)
// Needed for Java API usage
def this(payload: Array[Byte], routingKey: String, properties: BasicProperties) =
this(payload, routingKey, false, false, Some(properties))
// Needed for Java API usage
def this(payload: Array[Byte], routingKey: String, mandatory: Boolean, immediate: Boolean, properties: BasicProperties) =
this(payload, routingKey, mandatory, immediate, Some(properties))
}
case class Delivery(
payload: Array[Byte],
routingKey: String,
deliveryTag: Long,
isRedeliver: Boolean,
properties: BasicProperties,
sender: Option[ActorRef]) extends AMQPMessage
// connection messages
case object Connect extends AMQPMessage
case object Connected extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object Reconnecting extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object Disconnected extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object ChannelRequest extends InternalAMQPMessage
// channel messages
case object Start extends AMQPMessage
case object Started extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object Restarting extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object Stopped extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
// delivery messages
case class Acknowledge(deliveryTag: Long) extends AMQPMessage
case class Acknowledged(deliveryTag: Long) extends AMQPMessage
case class Reject(deliveryTag: Long) extends AMQPMessage
case class Rejected(deliveryTag: Long) extends AMQPMessage
class RejectionException(deliveryTag: Long) extends RuntimeException
// internal messages
private[akka] case class Failure(cause: Throwable) extends InternalAMQPMessage
case class ConnectionShutdown(cause: ShutdownSignalException) extends InternalAMQPMessage
case class ChannelShutdown(cause: ShutdownSignalException) extends InternalAMQPMessage
private[akka] class MessageNotDeliveredException(
val message: String,
val replyCode: Int,
val replyText: String,
val exchange: String,
val routingKey: String,
val properties: BasicProperties,
val body: Array[Byte]) extends RuntimeException(message)

View file

@ -1,143 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
import collection.JavaConversions
import akka.util.Logging
import com.rabbitmq.client.AMQP.BasicProperties
import com.rabbitmq.client.{Channel, Envelope, DefaultConsumer}
import akka.amqp.AMQP._
private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters)
extends FaultTolerantChannelActor(
consumerParameters.exchangeParameters, consumerParameters.channelParameters) {
import consumerParameters._
var listenerTag: Option[String] = None
def specificMessageHandler = {
case Acknowledge(deliveryTag) => acknowledgeDeliveryTag(deliveryTag, true)
case Reject(deliveryTag) => rejectDeliveryTag(deliveryTag, true)
case message: Message =>
handleIllegalMessage("%s can't be used to send messages, ignoring message [%s]".format(this, message))
case unknown =>
handleIllegalMessage("Unknown message [%s] to %s".format(unknown, this))
}
protected def setupChannel(ch: Channel) = {
channelParameters.foreach(params => ch.basicQos(params.prefetchSize))
val exchangeName = exchangeParameters.flatMap(params => Some(params.exchangeName))
val consumingQueue = exchangeName match {
case Some(exchange) =>
val queueDeclare: com.rabbitmq.client.AMQP.Queue.DeclareOk = {
queueName match {
case Some(name) =>
declareQueue(ch, name, queueDeclaration)
case None =>
log.debug("Declaring new generated queue for %s", toString)
ch.queueDeclare
}
}
log.debug("Binding new queue [%s] with [%s] for %s", queueDeclare.getQueue, routingKey, toString)
ch.queueBind(queueDeclare.getQueue, exchange, routingKey)
queueDeclare.getQueue
case None =>
// no exchange, use routing key as queuename
log.debug("No exchange specified, creating queue using routingkey as name (%s)", routingKey)
declareQueue(ch, routingKey, queueDeclaration)
routingKey
}
val tag = ch.basicConsume(consumingQueue, false, new DefaultConsumer(ch) with Logging {
override def handleDelivery(tag: String, envelope: Envelope, properties: BasicProperties, payload: Array[Byte]) {
try {
val deliveryTag = envelope.getDeliveryTag
log.debug("Passing a message on to %s", toString)
import envelope._
deliveryHandler ! Delivery(payload, getRoutingKey, getDeliveryTag, isRedeliver, properties, someSelf)
if (selfAcknowledging) {
log.debug("Self acking...")
acknowledgeDeliveryTag(deliveryTag, false)
}
} catch {
case cause =>
log.error(cause, "Delivery of message to %s failed", toString)
self ! Failure(cause) // pass on and re-throw exception in consumer actor to trigger restart and connect
}
}
})
listenerTag = Some(tag)
log.info("Intitialized %s", toString)
}
private def declareQueue(ch: Channel, queueName: String, queueDeclaration: Declaration): com.rabbitmq.client.AMQP.Queue.DeclareOk = {
queueDeclaration match {
case PassiveDeclaration =>
log.debug("Passively declaring new queue [%s] for %s", queueName, toString)
ch.queueDeclarePassive(queueName)
case ActiveDeclaration(durable, autoDelete, exclusive) =>
log.debug("Actively declaring new queue [%s] for %s", queueName, toString)
val configurationArguments = exchangeParameters match {
case Some(params) => params.configurationArguments
case _ => Map.empty
}
ch.queueDeclare(queueName, durable, exclusive, autoDelete, JavaConversions.asJavaMap(configurationArguments.toMap))
case NoActionDeclaration => new com.rabbitmq.client.impl.AMQImpl.Queue.DeclareOk(queueName, 0, 0) // do nothing here
}
}
private def acknowledgeDeliveryTag(deliveryTag: Long, remoteAcknowledgement: Boolean) = {
log.debug("Acking message with delivery tag [%s]", deliveryTag)
channel.foreach {
ch =>
ch.basicAck(deliveryTag, false)
if (remoteAcknowledgement) {
deliveryHandler ! Acknowledged(deliveryTag)
}
}
}
private def rejectDeliveryTag(deliveryTag: Long, remoteAcknowledgement: Boolean) = {
log.debug("Rejecting message with delivery tag [%s]", deliveryTag)
// FIXME: when rabbitmq 1.9 arrives, basicReject should be available on the API and implemented instead of this
log.warning("Consumer is rejecting delivery with tag [%s] - " +
"for now this means we have to self terminate and kill the channel - see you in a second.")
channel.foreach {
ch =>
if (remoteAcknowledgement) {
deliveryHandler ! Rejected(deliveryTag)
}
}
throw new RejectionException(deliveryTag)
}
private def handleIllegalMessage(errorMessage: String) = {
log.error(errorMessage)
throw new IllegalArgumentException(errorMessage)
}
override def preRestart(reason: Throwable) = {
listenerTag = None
super.preRestart(reason)
}
override def postStop = {
listenerTag.foreach(tag => channel.foreach(_.basicCancel(tag)))
self.shutdownLinkedActors
super.postStop
}
override def toString =
"AMQP.Consumer[id= " + self.id +
", exchangeParameters=" + exchangeParameters +
", queueDeclaration=" + queueDeclaration + "]"
}

View file

@ -1,268 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
import rpc.RPC
import rpc.RPC.{RpcClientSerializer, RpcServerSerializer}
import akka.actor.{Actor, ActorRegistry}
import Actor._
import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.lang.String
import akka.amqp.AMQP._
import akka.remote.protocol.RemoteProtocol.AddressProtocol
object ExampleSession {
def main(args: Array[String]) = {
printTopic("DIRECT")
direct
printTopic("FANOUT")
fanout
printTopic("TOPIC")
topic
printTopic("CALLBACK")
callback
printTopic("EASY STRING PRODUCER AND CONSUMER")
easyStringProducerConsumer
printTopic("EASY PROTOBUF PRODUCER AND CONSUMER")
easyProtobufProducerConsumer
printTopic("RPC")
rpc
printTopic("EASY STRING RPC")
easyStringRpc
printTopic("EASY PROTOBUF RPC")
easyProtobufRpc
printTopic("Happy hAkking :-)")
// postStop everything the amqp tree except the main AMQP supervisor
// all connections/consumers/producers will be stopped
AMQP.shutdownAll
ActorRegistry.shutdownAll
System.exit(0)
}
def printTopic(topic: String) {
println("")
println("==== " + topic + " ===")
println("")
TimeUnit.SECONDS.sleep(2)
}
def direct = {
// defaults to amqp://guest:guest@localhost:5672/
val connection = AMQP.newConnection()
val exchangeParameters = ExchangeParameters("my_direct_exchange", Direct)
val consumer = AMQP.newConsumer(connection, ConsumerParameters("some.routing", actorOf(new Actor { def receive = {
case Delivery(payload, _, _, _, _, _) => log.info("@george_bush received message from: %s", new String(payload))
}}), None, Some(exchangeParameters)))
val producer = AMQP.newProducer(connection, ProducerParameters(Some(exchangeParameters)))
producer ! Message("@jonas_boner: You sucked!!".getBytes, "some.routing")
}
def fanout = {
// defaults to amqp://guest:guest@localhost:5672/
val connection = AMQP.newConnection()
val exchangeParameters = ExchangeParameters("my_fanout_exchange", Fanout)
val bushConsumer = AMQP.newConsumer(connection, ConsumerParameters("@george_bush", actorOf(new Actor { def receive = {
case Delivery(payload, _, _, _, _, _) => log.info("@george_bush received message from: %s", new String(payload))
}}), None, Some(exchangeParameters)))
val obamaConsumer = AMQP.newConsumer(connection, ConsumerParameters("@barack_obama", actorOf(new Actor { def receive = {
case Delivery(payload, _, _, _, _, _) => log.info("@barack_obama received message from: %s", new String(payload))
}}), None, Some(exchangeParameters)))
val producer = AMQP.newProducer(connection, ProducerParameters(Some(exchangeParameters)))
producer ! Message("@jonas_boner: I'm going surfing".getBytes, "")
}
def topic = {
// defaults to amqp://guest:guest@localhost:5672/
val connection = AMQP.newConnection()
val exchangeParameters = ExchangeParameters("my_topic_exchange", Topic)
val bushConsumer = AMQP.newConsumer(connection, ConsumerParameters("@george_bush", actorOf(new Actor { def receive = {
case Delivery(payload, _, _, _, _, _) => log.info("@george_bush received message from: %s", new String(payload))
}}), None, Some(exchangeParameters)))
val obamaConsumer = AMQP.newConsumer(connection, ConsumerParameters("@barack_obama", actorOf(new Actor { def receive = {
case Delivery(payload, _, _, _, _, _) => log.info("@barack_obama received message from: %s", new String(payload))
}}), None, Some(exchangeParameters)))
val producer = AMQP.newProducer(connection, ProducerParameters(Some(exchangeParameters)))
producer ! Message("@jonas_boner: You still suck!!".getBytes, "@george_bush")
producer ! Message("@jonas_boner: Yes I can!".getBytes, "@barack_obama")
}
def callback = {
val channelCountdown = new CountDownLatch(2)
val connectionCallback = actorOf(new Actor { def receive = {
case Connected => log.info("Connection callback: Connected!")
case Reconnecting => () // not used, sent when connection fails and initiates a reconnect
case Disconnected => log.info("Connection callback: Disconnected!")
}})
val connection = AMQP.newConnection(new ConnectionParameters(connectionCallback = Some(connectionCallback)))
val channelCallback = actorOf(new Actor { def receive = {
case Started => {
log.info("Channel callback: Started")
channelCountdown.countDown
}
case Restarting => // not used, sent when channel or connection fails and initiates a restart
case Stopped => log.info("Channel callback: Stopped")
}})
val exchangeParameters = ExchangeParameters("my_callback_exchange", Direct)
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val consumer = AMQP.newConsumer(connection, ConsumerParameters("callback.routing", actorOf(new Actor { def receive = {
case _ => () // not used
}}), None, Some(exchangeParameters), channelParameters = Some(channelParameters)))
val producer = AMQP.newProducer(connection, ProducerParameters(Some(exchangeParameters)))
// Wait until both channels (producer & consumer) are started before stopping the connection
channelCountdown.await(2, TimeUnit.SECONDS)
connection.stop
}
def easyStringProducerConsumer = {
val connection = AMQP.newConnection()
val exchangeName = "easy.string"
// listen by default to:
// exchange = optional exchangeName
// routingKey = provided routingKey or <exchangeName>.request
// queueName = <routingKey>.in
AMQP.newStringConsumer(connection, message => println("Received message: "+message), Some(exchangeName))
// send by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
val producer = AMQP.newStringProducer(connection, Some(exchangeName))
producer.send("This shit is easy!")
}
def easyProtobufProducerConsumer = {
val connection = AMQP.newConnection()
val exchangeName = "easy.protobuf"
def protobufMessageHandler(message: AddressProtocol) = {
log.info("Received "+message)
}
AMQP.newProtobufConsumer(connection, protobufMessageHandler _, Some(exchangeName))
val producerClient = AMQP.newProtobufProducer[AddressProtocol](connection, Some(exchangeName))
producerClient.send(AddressProtocol.newBuilder.setHostname("akkarocks.com").setPort(1234).build)
}
def rpc = {
val connection = AMQP.newConnection()
val exchangeName = "my_rpc_exchange"
/** Server */
val serverFromBinary = new FromBinary[String] {
def fromBinary(bytes: Array[Byte]) = new String(bytes)
}
val serverToBinary = new ToBinary[Int] {
def toBinary(t: Int) = Array(t.toByte)
}
val rpcServerSerializer = new RpcServerSerializer[String, Int](serverFromBinary, serverToBinary)
def requestHandler(request: String) = 3
val rpcServer = RPC.newRpcServer(connection, exchangeName, rpcServerSerializer, requestHandler _,
routingKey = Some("rpc.in.key"), queueName = Some("rpc.in.key.queue"))
/** Client */
val clientToBinary = new ToBinary[String] {
def toBinary(t: String) = t.getBytes
}
val clientFromBinary = new FromBinary[Int] {
def fromBinary(bytes: Array[Byte]) = bytes.head.toInt
}
val rpcClientSerializer = new RpcClientSerializer[String, Int](clientToBinary, clientFromBinary)
val rpcClient = RPC.newRpcClient(connection, exchangeName, rpcClientSerializer, Some("rpc.in.key"))
val response = rpcClient.call("rpc_request")
log.info("Response: " + response)
}
def easyStringRpc = {
val connection = AMQP.newConnection()
val exchangeName = "easy.stringrpc"
// listen by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
// queueName = <routingKey>.in
RPC.newStringRpcServer(connection, exchangeName, request => {
log.info("Got request: "+request)
"Response to: '"+request+"'"
})
// send by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
val stringRpcClient = RPC.newStringRpcClient(connection, exchangeName)
val response = stringRpcClient.call("AMQP Rocks!")
log.info("Got response: "+response)
stringRpcClient.callAsync("AMQP is dead easy") {
case response => log.info("This is handled async: "+response)
}
}
def easyProtobufRpc = {
val connection = AMQP.newConnection()
val exchangeName = "easy.protobuf.rpc"
def protobufRequestHandler(request: AddressProtocol): AddressProtocol = {
AddressProtocol.newBuilder.setHostname(request.getHostname.reverse).setPort(request.getPort).build
}
RPC.newProtobufRpcServer(connection, exchangeName, protobufRequestHandler)
val protobufRpcClient = RPC.newProtobufRpcClient[AddressProtocol, AddressProtocol](connection, exchangeName)
val response = protobufRpcClient.call(AddressProtocol.newBuilder.setHostname("localhost").setPort(4321).build)
log.info("Got response: "+response)
}
}

View file

@ -1,23 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
sealed trait ExchangeType
case object Direct extends ExchangeType {
def getInstance() = this // Needed for Java API usage
override def toString = "direct"
}
case object Topic extends ExchangeType {
def getInstance() = this // Needed for Java API usage
override def toString = "topic"
}
case object Fanout extends ExchangeType {
def getInstance() = this // Needed for Java API usage
override def toString = "fanout"
}
case object Match extends ExchangeType {
def getInstance() = this // Needed for Java API usage
override def toString = "match"
}

View file

@ -1,108 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
import collection.JavaConversions
import java.lang.Throwable
import akka.actor.Actor
import Actor._
import com.rabbitmq.client.{ShutdownSignalException, Channel, ShutdownListener}
import scala.PartialFunction
import akka.amqp.AMQP._
abstract private[amqp] class FaultTolerantChannelActor(
exchangeParameters: Option[ExchangeParameters], channelParameters: Option[ChannelParameters]) extends Actor {
protected[amqp] var channel: Option[Channel] = None
log.info("%s is started", toString)
override def receive = channelMessageHandler orElse specificMessageHandler
// to be defined in subclassing actor
def specificMessageHandler: PartialFunction[Any, Unit]
private def channelMessageHandler: PartialFunction[Any, Unit] = {
case Start =>
// ask the connection for a new channel
self.supervisor.foreach {
sup =>
log.info("%s is requesting new channel from supervising connection", toString)
val newChannel: Option[Option[Channel]] = (sup !! ChannelRequest).as[Option[Channel]]
newChannel.foreach(ch => ch.foreach(c => setupChannelInternal(c)))
}
case ch: Channel => {
setupChannelInternal(ch)
}
case ChannelShutdown(cause) => {
closeChannel
if (cause.isHardError) {
// connection error
if (cause.isInitiatedByApplication) {
log.info("%s got normal shutdown", toString)
} else {
log.error(cause, "%s got hard error", toString)
}
} else {
// channel error
log.error(cause, "%s self restarting because of channel shutdown", toString)
notifyCallback(Restarting)
self ! Start
}
}
case Failure(cause) =>
log.error(cause, "%s self restarting because of channel failure", toString)
closeChannel
notifyCallback(Restarting)
self ! Start
}
// to be defined in subclassing actor
protected def setupChannel(ch: Channel)
private def setupChannelInternal(ch: Channel) = if (channel.isEmpty) {
exchangeParameters.foreach {
params =>
import params._
exchangeDeclaration match {
case PassiveDeclaration => ch.exchangeDeclarePassive(exchangeName)
case ActiveDeclaration(durable, autoDelete, _) =>
ch.exchangeDeclare(exchangeName, exchangeType.toString, durable, autoDelete, JavaConversions.asJavaMap(configurationArguments))
case NoActionDeclaration => // ignore
}
}
ch.addShutdownListener(new ShutdownListener {
def shutdownCompleted(cause: ShutdownSignalException) = {
self ! ChannelShutdown(cause)
}
})
channelParameters.foreach(_.shutdownListener.foreach(sdl => ch.getConnection.addShutdownListener(sdl)))
setupChannel(ch)
channel = Some(ch)
notifyCallback(Started)
log.info("Channel setup for %s", toString)
}
private def closeChannel = {
channel.foreach {
ch =>
if (ch.isOpen) ch.close
notifyCallback(Stopped)
log.info("%s channel closed", toString)
}
channel = None
}
private def notifyCallback(message: AMQPMessage) = {
channelParameters.foreach(_.channelCallback.foreach(cb => if (cb.isRunning) cb ! message))
}
override def preRestart(reason: Throwable) = {
notifyCallback(Restarting)
closeChannel
}
override def postStop = closeChannel
}

View file

@ -1,118 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
import java.util.{TimerTask, Timer}
import java.io.IOException
import com.rabbitmq.client._
import akka.amqp.AMQP.ConnectionParameters
import akka.config.Supervision.{ Permanent, OneForOneStrategy }
import akka.actor.{Exit, Actor}
private[amqp] class FaultTolerantConnectionActor(connectionParameters: ConnectionParameters) extends Actor {
import connectionParameters._
self.id = "amqp-connection-%s".format(host)
self.lifeCycle = Permanent
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]))
val reconnectionTimer = new Timer("%s-timer".format(self.id))
val connectionFactory: ConnectionFactory = new ConnectionFactory()
connectionFactory.setHost(host)
connectionFactory.setPort(port)
connectionFactory.setUsername(username)
connectionFactory.setPassword(password)
connectionFactory.setVirtualHost(virtualHost)
var connection: Option[Connection] = None
protected def receive = {
case Connect => connect
case ChannelRequest => {
connection match {
case Some(conn) => {
val chanel: Channel = conn.createChannel
self.reply(Some(chanel))
}
case None => {
log.warning("Unable to create new channel - no connection")
self.reply(None)
}
}
}
case ConnectionShutdown(cause) => {
if (cause.isHardError) {
// connection error
if (cause.isInitiatedByApplication) {
log.info("ConnectionShutdown by application [%s]", self.id)
} else {
log.error(cause, "ConnectionShutdown is hard error - self terminating")
self ! new Exit(self, cause)
}
}
}
}
private def connect = if (connection.isEmpty || !connection.get.isOpen) {
try {
connection = Some(connectionFactory.newConnection)
connection.foreach {
conn =>
conn.addShutdownListener(new ShutdownListener {
def shutdownCompleted(cause: ShutdownSignalException) = {
self ! ConnectionShutdown(cause)
}
})
log.info("Successfully (re)connected to AMQP Server %s:%s [%s]", host, port, self.id)
log.debug("Sending new channel to %d already linked actors", self.linkedActors.size)
import scala.collection.JavaConversions._
self.linkedActors.values.iterator.foreach(_ ! conn.createChannel)
notifyCallback(Connected)
}
} catch {
case e: Exception =>
connection = None
log.info("Trying to connect to AMQP server in %d milliseconds [%s]"
, connectionParameters.initReconnectDelay, self.id)
reconnectionTimer.schedule(new TimerTask() {
override def run = {
notifyCallback(Reconnecting)
self ! Connect
}
}, connectionParameters.initReconnectDelay)
}
}
private def disconnect = {
try {
connection.foreach(_.close)
log.debug("Disconnected AMQP connection at %s:%s [%s]", host, port, self.id)
notifyCallback(Disconnected)
} catch {
case e: IOException => log.error("Could not close AMQP connection %s:%s [%s]", host, port, self.id)
case _ => ()
}
connection = None
}
private def notifyCallback(message: AMQPMessage) = {
connectionCallback.foreach(cb => if (cb.isRunning) cb ! message)
}
override def postStop = {
reconnectionTimer.cancel
// make sure postStop is called on all linked actors so they can do channel cleanup before connection is killed
self.shutdownLinkedActors
disconnect
}
override def preRestart(reason: Throwable) = disconnect
override def postRestart(reason: Throwable) = {
notifyCallback(Reconnecting)
connect
}
}

View file

@ -1,60 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
import com.rabbitmq.client._
import akka.amqp.AMQP.ProducerParameters
private[amqp] class ProducerActor(producerParameters: ProducerParameters)
extends FaultTolerantChannelActor(
producerParameters.exchangeParameters, producerParameters.channelParameters) {
import producerParameters._
val exchangeName = exchangeParameters.flatMap(params => Some(params.exchangeName))
producerId.foreach(id => self.id = id)
def specificMessageHandler = {
case message@Message(payload, routingKey, mandatory, immediate, properties) if channel.isDefined => {
log.debug("Sending message [%s]", message)
channel.foreach(_.basicPublish(exchangeName.getOrElse(""), routingKey, mandatory, immediate, properties.getOrElse(null), payload))
}
case message@Message(payload, routingKey, mandatory, immediate, properties) => {
log.warning("Unable to send message [%s]", message)
// FIXME: If channel is not available, messages should be queued back into the actor mailbox and actor should only react on 'Start'
}
}
protected def setupChannel(ch: Channel) {
returnListener match {
case Some(listener) => ch.setReturnListener(listener)
case None => ch.setReturnListener(new ReturnListener() {
def handleBasicReturn(
replyCode: Int,
replyText: String,
exchange: String,
routingKey: String,
properties: com.rabbitmq.client.AMQP.BasicProperties,
body: Array[Byte]) = {
throw new MessageNotDeliveredException(
"Could not deliver message [" + body +
"] with reply code [" + replyCode +
"] with reply text [" + replyText +
"] and routing key [" + routingKey +
"] to exchange [" + exchange + "]",
replyCode, replyText, exchange, routingKey, properties, body)
}
})
}
}
override def toString =
"AMQP.Poducer[id= "+ self.id +
", exchangeParameters=" + exchangeParameters + "]"
}

View file

@ -1,324 +0,0 @@
package akka.amqp.rpc
import akka.amqp.AMQP._
import com.google.protobuf.Message
import akka.actor.{Actor, ActorRef}
import Actor._
import akka.amqp._
import reflect.Manifest
import akka.japi
object RPC {
// Needed for Java API usage
def newRpcClient[O, I](connection: ActorRef,
exchangeName: String,
routingKey: String,
serializer: RpcClientSerializer[O, I]): RpcClient[O,I] = {
newRpcClient(connection, exchangeName, serializer, Some(routingKey), None)
}
// Needed for Java API usage
def newRpcClient[O, I](connection: ActorRef,
exchangeName: String,
routingKey: String,
serializer: RpcClientSerializer[O, I],
channelParameters: ChannelParameters): RpcClient[O,I] = {
newRpcClient(connection, exchangeName, serializer, Some(routingKey), Some(channelParameters))
}
def newRpcClient[O, I](connection: ActorRef,
exchangeName: String,
serializer: RpcClientSerializer[O, I],
routingKey: Option[String] = None,
channelParameters: Option[ChannelParameters] = None): RpcClient[O,I] = {
val rKey = routingKey.getOrElse("%s.request".format(exchangeName))
val rpcActor: ActorRef = actorOf(new RpcClientActor[O, I](
ExchangeParameters(exchangeName, exchangeDeclaration = PassiveDeclaration), rKey, serializer, channelParameters))
connection.startLink(rpcActor)
rpcActor ! Start
rpcActor
new RpcClient(rpcActor)
}
// Needed for Java API usage
def newRpcServer[I, O](connection: ActorRef,
exchangeName: String,
serializer: RpcServerSerializer[I, O],
requestHandler: japi.Function[I,O],
routingKey: String): RpcServerHandle = {
newRpcServer(connection, exchangeName, serializer, requestHandler.apply _, Some(routingKey))
}
// Needed for Java API usage
def newRpcServer[I, O](connection: ActorRef,
exchangeName: String,
serializer: RpcServerSerializer[I, O],
requestHandler: Function[I,O],
routingKey: String,
queueName: String): RpcServerHandle = {
newRpcServer(connection, exchangeName, serializer, requestHandler.apply _, Some(routingKey), Some(queueName))
}
// Needed for Java API usage
def newRpcServer[I, O](connection: ActorRef,
exchangeName: String,
serializer: RpcServerSerializer[I, O],
requestHandler: japi.Function[I,O],
routingKey: String,
channelParameters: ChannelParameters): RpcServerHandle = {
newRpcServer(connection, exchangeName, serializer, requestHandler.apply _, Some(routingKey), None, Some(channelParameters))
}
// Needed for Java API usage
def newRpcServer[I, O](connection: ActorRef,
exchangeName: String,
serializer: RpcServerSerializer[I, O],
requestHandler: japi.Function[I,O],
routingKey: String,
queueName: String,
channelParameters: ChannelParameters): RpcServerHandle = {
newRpcServer(connection, exchangeName, serializer, requestHandler.apply _, Some(routingKey), Some(queueName), Some(channelParameters))
}
def newRpcServer[I, O](connection: ActorRef,
exchangeName: String,
serializer: RpcServerSerializer[I, O],
requestHandler: I => O,
routingKey: Option[String] = None,
queueName: Option[String] = None,
channelParameters: Option[ChannelParameters] = None,
poolSize: Int = 1): RpcServerHandle = {
val rKey = routingKey.getOrElse("%s.request".format(exchangeName))
val qName = queueName.getOrElse("%s.in".format(rKey))
val producer = newProducer(connection, ProducerParameters(channelParameters = channelParameters))
val consumers = (1 to poolSize).map {
num =>
val rpcServer = actorOf(new RpcServerActor[I, O](producer, serializer, requestHandler))
newConsumer(connection, ConsumerParameters(rKey, rpcServer,
exchangeParameters = Some(ExchangeParameters(exchangeName)), channelParameters = channelParameters,
selfAcknowledging = false, queueName = Some(qName)))
}
RpcServerHandle(producer, consumers)
}
case class RpcServerHandle(producer: ActorRef, consumers: Seq[ActorRef]) {
def stop = {
consumers.foreach(_.stop)
producer.stop
}
}
case class RpcClientSerializer[O, I](toBinary: ToBinary[O], fromBinary: FromBinary[I])
case class RpcServerSerializer[I, O](fromBinary: FromBinary[I], toBinary: ToBinary[O])
/**
* RPC convenience
*/
class RpcClient[O, I](client: ActorRef){
// Needed for Java API usage
def call(request: O): Option[I] = {
call(request, 5000)
}
def call(request: O, timeout: Long = 5000): Option[I] = {
(client.!!(request, timeout)).as[I]
}
// Needed for Java API usage
def callAsync(request: O, responseHandler: japi.Procedure[I]): Unit = {
callAsync(request, 5000, responseHandler)
}
// Needed for Java API usage
def callAsync(request: O, timeout: Long, responseHandler: japi.Procedure[I]): Unit = {
callAsync(request, timeout){
case Some(response) => responseHandler.apply(response)
}
}
def callAsync(request: O, timeout: Long = 5000)(responseHandler: PartialFunction[Option[I],Unit]) = {
spawn {
val result = call(request, timeout)
responseHandler.apply(result)
}
}
def stop = client.stop
}
// Needed for Java API usage
def newProtobufRpcServer[I <: Message, O <: Message](
connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[I,O],
resultClazz: Class[I]): RpcServerHandle = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcServer(connection, exchangeName, requestHandler.apply _)
}
// Needed for Java API usage
def newProtobufRpcServer[I <: Message, O <: Message](
connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[I,O],
routingKey: String,
resultClazz: Class[I]): RpcServerHandle = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcServer(connection, exchangeName, requestHandler.apply _, Some(routingKey))
}
// Needed for Java API usage
def newProtobufRpcServer[I <: Message, O <: Message](
connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[I,O],
routingKey: String,
queueName: String,
resultClazz: Class[I]): RpcServerHandle = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcServer(connection, exchangeName, requestHandler.apply _, Some(routingKey), Some(queueName))
}
def newProtobufRpcServer[I <: Message, O <: Message](
connection: ActorRef,
exchangeName: String,
requestHandler: I => O,
routingKey: Option[String] = None,
queueName: Option[String] = None)(implicit manifest: Manifest[I]): RpcServerHandle = {
val serializer = new RpcServerSerializer[I, O](
new FromBinary[I] {
def fromBinary(bytes: Array[Byte]): I = {
createProtobufFromBytes[I](bytes)
}
}, new ToBinary[O] {
def toBinary(t: O) = t.toByteArray
})
newRpcServer(connection, exchangeName, serializer, requestHandler, routingKey, queueName)
}
// Needed for Java API usage
def newProtobufRpcClient[O <: Message, I <: Message](
connection: ActorRef,
exchangeName: String,
resultClazz: Class[I]): RpcClient[O, I] = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcClient(connection, exchangeName, None)
}
// Needed for Java API usage
def newProtobufRpcClient[O <: Message, I <: Message](
connection: ActorRef,
exchangeName: String,
routingKey: String,
resultClazz: Class[I]): RpcClient[O, I] = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcClient(connection, exchangeName, Some(routingKey))
}
def newProtobufRpcClient[O <: Message, I <: Message](
connection: ActorRef,
exchangeName: String,
routingKey: Option[String] = None)(implicit manifest: Manifest[I]): RpcClient[O, I] = {
val serializer = new RpcClientSerializer[O, I](
new ToBinary[O] {
def toBinary(t: O) = t.toByteArray
}, new FromBinary[I] {
def fromBinary(bytes: Array[Byte]): I = {
createProtobufFromBytes[I](bytes)
}
})
newRpcClient(connection, exchangeName, serializer, routingKey)
}
// Needed for Java API usage
def newStringRpcServer(connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[String,String]): RpcServerHandle = {
newStringRpcServer(connection, exchangeName, requestHandler.apply _)
}
// Needed for Java API usage
def newStringRpcServer(connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[String,String],
routingKey: String): RpcServerHandle = {
newStringRpcServer(connection, exchangeName, requestHandler.apply _, Some(routingKey))
}
// Needed for Java API usage
def newStringRpcServer(connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[String,String],
routingKey: String,
queueName: String): RpcServerHandle = {
newStringRpcServer(connection, exchangeName, requestHandler.apply _, Some(routingKey), Some(queueName))
}
def newStringRpcServer(connection: ActorRef,
exchangeName: String,
requestHandler: String => String,
routingKey: Option[String] = None,
queueName: Option[String] = None): RpcServerHandle = {
val serializer = new RpcServerSerializer[String, String](
new FromBinary[String] {
def fromBinary(bytes: Array[Byte]): String = {
new String(bytes)
}
}, new ToBinary[String] {
def toBinary(t: String) = t.getBytes
})
newRpcServer(connection, exchangeName, serializer, requestHandler, routingKey, queueName)
}
// Needed for Java API usage
def newStringRpcClient(connection: ActorRef,
exchange: String): RpcClient[String, String] = {
newStringRpcClient(connection, exchange, None)
}
// Needed for Java API usage
def newStringRpcClient(connection: ActorRef,
exchange: String,
routingKey: String): RpcClient[String, String] = {
newStringRpcClient(connection, exchange, Some(routingKey))
}
def newStringRpcClient(connection: ActorRef,
exchange: String,
routingKey: Option[String] = None): RpcClient[String, String] = {
val serializer = new RpcClientSerializer[String, String](
new ToBinary[String] {
def toBinary(t: String) = t.getBytes
}, new FromBinary[String] {
def fromBinary(bytes: Array[Byte]): String = {
new String(bytes)
}
})
newRpcClient(connection, exchange, serializer, routingKey)
}
}

View file

@ -1,49 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
import com.rabbitmq.client.{Channel, RpcClient}
import rpc.RPC.RpcClientSerializer
import akka.amqp.AMQP.{ChannelParameters, ExchangeParameters}
class RpcClientActor[I,O](
exchangeParameters: ExchangeParameters,
routingKey: String,
serializer: RpcClientSerializer[I,O],
channelParameters: Option[ChannelParameters] = None)
extends FaultTolerantChannelActor(Some(exchangeParameters), channelParameters) {
import exchangeParameters._
var rpcClient: Option[RpcClient] = None
log.info("%s started", this)
def specificMessageHandler = {
case payload: I => {
rpcClient match {
case Some(client) =>
val response: Array[Byte] = client.primitiveCall(serializer.toBinary.toBinary(payload))
self.reply(serializer.fromBinary.fromBinary(response))
case None => error("%s has no client to send messages with".format(this))
}
}
}
protected def setupChannel(ch: Channel) = rpcClient = Some(new RpcClient(ch, exchangeName, routingKey))
override def preRestart(reason: Throwable) = {
rpcClient = None
super.preRestart(reason)
}
override def postStop = {
rpcClient.foreach(rpc => rpc.close)
super.postStop
}
override def toString = "AMQP.RpcClient[exchange=" +exchangeName + ", routingKey=" + routingKey+ "]"
}

View file

@ -1,36 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp
import rpc.RPC.RpcServerSerializer
import akka.actor.{ActorRef, Actor}
import com.rabbitmq.client.AMQP.BasicProperties
class RpcServerActor[I,O](
producer: ActorRef,
serializer: RpcServerSerializer[I,O],
requestHandler: I => O) extends Actor {
log.info("%s started", this)
protected def receive = {
case Delivery(payload, _, tag, _, props, sender) => {
log.debug("%s handling delivery with tag %d", this, tag)
val request = serializer.fromBinary.fromBinary(payload)
val response: Array[Byte] = serializer.toBinary.toBinary(requestHandler(request))
log.debug("%s sending reply to %s", this, props.getReplyTo)
val replyProps = new BasicProperties
replyProps.setCorrelationId(props.getCorrelationId)
producer ! new Message(response, props.getReplyTo, properties = Some(replyProps))
sender.foreach(_ ! Acknowledge(tag))
}
case Acknowledged(tag) => log.debug("%s acknowledged delivery with tag %d", this, tag)
}
override def toString = "AMQP.RpcServer[]"
}

View file

@ -1,54 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import java.util.concurrent.TimeUnit
import akka.actor.{Actor, ActorRef}
import org.multiverse.api.latches.StandardLatch
import com.rabbitmq.client.ShutdownSignalException
import akka.amqp._
import akka.amqp.AMQP.ConnectionParameters
import org.scalatest.matchers.MustMatchers
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPConnectionRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def connectionAndRecovery = AMQPTest.withCleanEndState {
val connectedLatch = new StandardLatch
val reconnectingLatch = new StandardLatch
val reconnectedLatch = new StandardLatch
val disconnectedLatch = new StandardLatch
val connectionCallback: ActorRef = Actor.actorOf( new Actor {
def receive = {
case Connected =>
if (!connectedLatch.isOpen) {
connectedLatch.open
} else {
reconnectedLatch.open
}
case Reconnecting => reconnectingLatch.open
case Disconnected => disconnectedLatch.open
}
}).start
val connection = AMQP.newConnection(ConnectionParameters(initReconnectDelay = 50, connectionCallback = Some(connectionCallback)))
try {
connectedLatch.tryAwait(2, TimeUnit.SECONDS) must be(true)
connection ! new ConnectionShutdown(new ShutdownSignalException(true, false, "TestException", "TestRef"))
reconnectingLatch.tryAwait(2, TimeUnit.SECONDS) must be(true)
reconnectedLatch.tryAwait(2, TimeUnit.SECONDS) must be(true)
} finally {
AMQP.shutdownAll
disconnectedLatch.tryAwait(2, TimeUnit.SECONDS) must be(true)
}
}
}

View file

@ -1,65 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import org.multiverse.api.latches.StandardLatch
import com.rabbitmq.client.ShutdownSignalException
import akka.amqp._
import org.scalatest.matchers.MustMatchers
import java.util.concurrent.TimeUnit
import org.junit.Test
import akka.amqp.AMQP._
import org.scalatest.junit.JUnitSuite
import akka.actor.Actor._
import akka.actor.{Actor, ActorRef}
class AMQPConsumerChannelRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerChannelRecovery = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection(ConnectionParameters(initReconnectDelay = 50))
try {
val producer = AMQP.newProducer(connection, ProducerParameters(
Some(ExchangeParameters("text_exchange"))))
val consumerStartedLatch = new StandardLatch
val consumerRestartedLatch = new StandardLatch
val consumerChannelCallback: ActorRef = actorOf( new Actor {
def receive = {
case Started => {
if (!consumerStartedLatch.isOpen) {
consumerStartedLatch.open
} else {
consumerRestartedLatch.open
}
}
case Restarting => ()
case Stopped => ()
}
}).start
val payloadLatch = new StandardLatch
val consumerExchangeParameters = ExchangeParameters("text_exchange")
val consumerChannelParameters = ChannelParameters(channelCallback = Some(consumerChannelCallback))
val consumer = AMQP.newConsumer(connection, ConsumerParameters("non.interesting.routing.key", actorOf( new Actor {
def receive = { case Delivery(payload, _, _, _, _, _) => payloadLatch.open }
}),
exchangeParameters = Some(consumerExchangeParameters), channelParameters = Some(consumerChannelParameters)))
consumerStartedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
val listenerLatch = new StandardLatch
consumer ! new ChannelShutdown(new ShutdownSignalException(false, false, "TestException", "TestRef"))
consumerRestartedLatch.tryAwait(4, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "non.interesting.routing.key")
payloadLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
} finally {
connection.stop
}
}
}

View file

@ -1,86 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import org.multiverse.api.latches.StandardLatch
import com.rabbitmq.client.ShutdownSignalException
import akka.amqp._
import org.scalatest.matchers.MustMatchers
import java.util.concurrent.TimeUnit
import org.junit.Test
import akka.amqp.AMQP._
import org.scalatest.junit.JUnitSuite
import akka.actor.{Actor, ActorRef}
import Actor._
class AMQPConsumerConnectionRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerConnectionRecovery = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection(ConnectionParameters(initReconnectDelay = 50))
try {
val producerStartedLatch = new StandardLatch
val producerRestartedLatch = new StandardLatch
val producerChannelCallback: ActorRef = actorOf( new Actor {
def receive = {
case Started => {
if (!producerStartedLatch.isOpen) {
producerStartedLatch.open
} else {
producerRestartedLatch.open
}
}
case Restarting => ()
case Stopped => ()
}
}).start
val channelParameters = ChannelParameters(channelCallback = Some(producerChannelCallback))
val producer = AMQP.newProducer(connection, ProducerParameters(
Some(ExchangeParameters("text_exchange")), channelParameters = Some(channelParameters)))
producerStartedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
val consumerStartedLatch = new StandardLatch
val consumerRestartedLatch = new StandardLatch
val consumerChannelCallback: ActorRef = actorOf( new Actor {
def receive = {
case Started => {
if (!consumerStartedLatch.isOpen) {
consumerStartedLatch.open
} else {
consumerRestartedLatch.open
}
}
case Restarting => ()
case Stopped => ()
}
}).start
val payloadLatch = new StandardLatch
val consumerExchangeParameters = ExchangeParameters("text_exchange")
val consumerChannelParameters = ChannelParameters(channelCallback = Some(consumerChannelCallback))
val consumer = AMQP.newConsumer(connection, ConsumerParameters("non.interesting.routing.key", actorOf( new Actor {
def receive = { case Delivery(payload, _, _, _, _, _) => payloadLatch.open }
}), exchangeParameters = Some(consumerExchangeParameters), channelParameters = Some(consumerChannelParameters)))
consumerStartedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
val listenerLatch = new StandardLatch
connection ! new ConnectionShutdown(new ShutdownSignalException(true, false, "TestException", "TestRef"))
producerRestartedLatch.tryAwait(4, TimeUnit.SECONDS) must be (true)
consumerRestartedLatch.tryAwait(4, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "non.interesting.routing.key")
payloadLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
} finally {
connection.stop
}
}
}

View file

@ -1,65 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import akka.actor.Actor._
import org.scalatest.matchers.MustMatchers
import akka.amqp._
import org.junit.Test
import java.util.concurrent.{CountDownLatch, TimeUnit}
import org.multiverse.api.latches.StandardLatch
import org.scalatest.junit.JUnitSuite
import akka.amqp.AMQP._
import akka.actor.{Actor, ActorRef}
class AMQPConsumerManualAcknowledgeTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessageManualAcknowledge = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
try {
val countDown = new CountDownLatch(2)
val channelCallback = actorOf( new Actor {
def receive = {
case Started => countDown.countDown
case Restarting => ()
case Stopped => ()
}
}).start
val exchangeParameters = ExchangeParameters("text_exchange")
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val failLatch = new StandardLatch
val acknowledgeLatch = new StandardLatch
var deliveryTagCheck: Long = -1
val consumer:ActorRef = AMQP.newConsumer(connection, ConsumerParameters("manual.ack.this", actorOf( new Actor {
def receive = {
case Delivery(payload, _, deliveryTag, _, _, sender) => {
if (!failLatch.isOpen) {
failLatch.open
error("Make it fail!")
} else {
deliveryTagCheck = deliveryTag
sender.foreach(_ ! Acknowledge(deliveryTag))
}
}
case Acknowledged(deliveryTag) => if (deliveryTagCheck == deliveryTag) acknowledgeLatch.open
}
}), queueName = Some("self.ack.queue"), exchangeParameters = Some(exchangeParameters),
selfAcknowledging = false, channelParameters = Some(channelParameters),
queueDeclaration = ActiveDeclaration(autoDelete = false)))
val producer = AMQP.newProducer(connection,
ProducerParameters(Some(exchangeParameters), channelParameters = Some(channelParameters)))
countDown.await(2, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "manual.ack.this")
acknowledgeLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
} finally {
connection.stop
}
}
}

View file

@ -1,56 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import akka.actor.Actor._
import org.scalatest.matchers.MustMatchers
import akka.amqp._
import org.junit.Test
import java.util.concurrent.{CountDownLatch, TimeUnit}
import akka.amqp.AMQP.{ExchangeParameters, ConsumerParameters, ChannelParameters, ProducerParameters}
import org.multiverse.api.latches.StandardLatch
import org.scalatest.junit.JUnitSuite
import akka.actor.{Actor, ActorRef}
class AMQPConsumerManualRejectTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessageManualAcknowledge = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
try {
val countDown = new CountDownLatch(2)
val restartingLatch = new StandardLatch
val channelCallback = actorOf(new Actor {
def receive = {
case Started => countDown.countDown
case Restarting => restartingLatch.open
case Stopped => ()
}
}).start
val exchangeParameters = ExchangeParameters("text_exchange")
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val rejectedLatch = new StandardLatch
val consumer:ActorRef = AMQP.newConsumer(connection, ConsumerParameters("manual.reject.this", actorOf( new Actor {
def receive = {
case Delivery(payload, _, deliveryTag, _, _, sender) => sender.foreach(_ ! Reject(deliveryTag))
case Rejected(deliveryTag) => rejectedLatch.open
}
}), queueName = Some("self.reject.queue"), exchangeParameters = Some(exchangeParameters),
selfAcknowledging = false, channelParameters = Some(channelParameters)))
val producer = AMQP.newProducer(connection,
ProducerParameters(Some(exchangeParameters), channelParameters = Some(channelParameters)))
countDown.await(2, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "manual.reject.this")
rejectedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
restartingLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
} finally {
connection.stop
}
}
}

View file

@ -1,46 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import akka.amqp._
import org.multiverse.api.latches.StandardLatch
import akka.actor.Actor._
import org.scalatest.matchers.MustMatchers
import java.util.concurrent.{CountDownLatch, TimeUnit}
import akka.amqp.AMQP.{ExchangeParameters, ConsumerParameters, ChannelParameters, ProducerParameters}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
import akka.actor.Actor
class AMQPConsumerMessageTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
val countDown = new CountDownLatch(2)
val channelCallback = actorOf(new Actor {
def receive = {
case Started => countDown.countDown
case Restarting => ()
case Stopped => ()
}
}).start
val exchangeParameters = ExchangeParameters("text_exchange")
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val payloadLatch = new StandardLatch
val consumer = AMQP.newConsumer(connection, ConsumerParameters("non.interesting.routing.key", actorOf(new Actor {
def receive = { case Delivery(payload, _, _, _, _, _) => payloadLatch.open }
}), exchangeParameters = Some(exchangeParameters), channelParameters = Some(channelParameters)))
val producer = AMQP.newProducer(connection,
ProducerParameters(Some(exchangeParameters), channelParameters = Some(channelParameters)))
countDown.await(2, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "non.interesting.routing.key")
payloadLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
}
}

View file

@ -1,45 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import akka.amqp._
import org.multiverse.api.latches.StandardLatch
import akka.actor.Actor._
import org.scalatest.matchers.MustMatchers
import java.util.concurrent.{CountDownLatch, TimeUnit}
import akka.amqp.AMQP.{ConsumerParameters, ChannelParameters, ProducerParameters}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
import akka.actor.Actor
class AMQPConsumerPrivateQueueTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
val countDown = new CountDownLatch(2)
val channelCallback = actorOf(new Actor {
def receive = {
case Started => countDown.countDown
case Restarting => ()
case Stopped => ()
}
}).start
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val payloadLatch = new StandardLatch
val consumer = AMQP.newConsumer(connection, ConsumerParameters("my.private.routing.key", actorOf(new Actor {
def receive = { case Delivery(payload, _, _, _, _, _) => payloadLatch.open }
}), channelParameters = Some(channelParameters)))
val producer = AMQP.newProducer(connection,
ProducerParameters(channelParameters = Some(channelParameters)))
countDown.await(2, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "my.private.routing.key")
payloadLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
}
}

View file

@ -1,57 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import java.util.concurrent.TimeUnit
import akka.actor.{Actor, ActorRef}
import org.multiverse.api.latches.StandardLatch
import com.rabbitmq.client.ShutdownSignalException
import akka.amqp._
import org.scalatest.matchers.MustMatchers
import akka.amqp.AMQP.{ExchangeParameters, ChannelParameters, ProducerParameters, ConnectionParameters}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPProducerChannelRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def producerChannelRecovery = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection(ConnectionParameters(initReconnectDelay = 50))
try {
val startedLatch = new StandardLatch
val restartingLatch = new StandardLatch
val restartedLatch = new StandardLatch
val producerCallback: ActorRef = Actor.actorOf( new Actor {
def receive = {
case Started => {
if (!startedLatch.isOpen) {
startedLatch.open
} else {
restartedLatch.open
}
}
case Restarting => restartingLatch.open
case Stopped => ()
}
}).start
val channelParameters = ChannelParameters(channelCallback = Some(producerCallback))
val producerParameters = ProducerParameters(
Some(ExchangeParameters("text_exchange")), channelParameters = Some(channelParameters))
val producer = AMQP.newProducer(connection, producerParameters)
startedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
producer ! new ChannelShutdown(new ShutdownSignalException(false, false, "TestException", "TestRef"))
restartingLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
restartedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
} finally {
connection.stop
}
}
}

View file

@ -1,56 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import java.util.concurrent.TimeUnit
import akka.actor.{Actor, ActorRef}
import org.multiverse.api.latches.StandardLatch
import com.rabbitmq.client.ShutdownSignalException
import akka.amqp._
import org.scalatest.matchers.MustMatchers
import akka.amqp.AMQP.{ExchangeParameters, ChannelParameters, ProducerParameters, ConnectionParameters}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPProducerConnectionRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def producerConnectionRecovery = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection(ConnectionParameters(initReconnectDelay = 50))
try {
val startedLatch = new StandardLatch
val restartingLatch = new StandardLatch
val restartedLatch = new StandardLatch
val producerCallback: ActorRef = Actor.actorOf(new Actor{
def receive = {
case Started => {
if (!startedLatch.isOpen) {
startedLatch.open
} else {
restartedLatch.open
}
}
case Restarting => restartingLatch.open
case Stopped => ()
}
}).start
val channelParameters = ChannelParameters(channelCallback = Some(producerCallback))
val producerParameters = ProducerParameters(
Some(ExchangeParameters("text_exchange")), channelParameters = Some(channelParameters))
val producer = AMQP.newProducer(connection, producerParameters)
startedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
connection ! new ConnectionShutdown(new ShutdownSignalException(true, false, "TestException", "TestRef"))
restartingLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
restartedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
} finally {
connection.stop
}
}
}

View file

@ -1,43 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import java.util.concurrent.TimeUnit
import akka.actor.ActorRef
import org.multiverse.api.latches.StandardLatch
import akka.amqp._
import com.rabbitmq.client.ReturnListener
import com.rabbitmq.client.AMQP.BasicProperties
import java.lang.String
import org.scalatest.matchers.MustMatchers
import akka.amqp.AMQP.{ExchangeParameters, ProducerParameters}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPProducerMessageTestIntegration extends JUnitSuite with MustMatchers {
@Test
def producerMessage = AMQPTest.withCleanEndState {
val connection: ActorRef = AMQP.newConnection()
try {
val returnLatch = new StandardLatch
val returnListener = new ReturnListener {
def handleBasicReturn(replyCode: Int, replyText: String, exchange: String, routingKey: String, properties: BasicProperties, body: Array[Byte]) = {
returnLatch.open
}
}
val producerParameters = ProducerParameters(
Some(ExchangeParameters("text_exchange")), returnListener = Some(returnListener))
val producer = AMQP.newProducer(connection, producerParameters)
producer ! new Message("some_payload".getBytes, "non.interesing.routing.key", mandatory = true)
returnLatch.tryAwait(2, TimeUnit.SECONDS) must be(true)
} finally {
connection.stop
}
}
}

View file

@ -1,43 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import org.scalatest.matchers.MustMatchers
import org.scalatest.junit.JUnitSuite
import akka.amqp.AMQP
import org.junit.Test
import org.multiverse.api.latches.StandardLatch
import java.util.concurrent.TimeUnit
import akka.amqp.rpc.RPC
import akka.remote.protocol.RemoteProtocol.AddressProtocol
class AMQPProtobufProducerConsumerTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
val responseLatch = new StandardLatch
RPC.newProtobufRpcServer(connection, "protoexchange", requestHandler)
val request = AddressProtocol.newBuilder.setHostname("testhost").setPort(4321).build
def responseHandler(response: AddressProtocol) = {
assert(response.getHostname == request.getHostname.reverse)
responseLatch.open
}
AMQP.newProtobufConsumer(connection, responseHandler _, None, Some("proto.reply.key"))
val producer = AMQP.newProtobufProducer[AddressProtocol](connection, Some("protoexchange"))
producer.send(request, Some("proto.reply.key"))
responseLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
}
def requestHandler(request: AddressProtocol): AddressProtocol = {
AddressProtocol.newBuilder.setHostname(request.getHostname.reverse).setPort(request.getPort).build
}
}

View file

@ -1,64 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import akka.amqp._
import rpc.RPC
import rpc.RPC.{RpcClientSerializer, RpcServerSerializer}
import akka.actor.Actor._
import org.scalatest.matchers.MustMatchers
import java.util.concurrent.{CountDownLatch, TimeUnit}
import akka.amqp.AMQP._
import org.scalatest.junit.JUnitSuite
import org.junit.Test
import akka.actor.Actor
class AMQPRpcClientServerTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
val countDown = new CountDownLatch(3)
val channelCallback = actorOf( new Actor {
def receive = {
case Started => countDown.countDown
case Restarting => ()
case Stopped => ()
}
}).start
val exchangeName = "text_topic_exchange"
val channelParameters = ChannelParameters(channelCallback
= Some(channelCallback))
val rpcServerSerializer = new RpcServerSerializer[String, Int](
new FromBinary[String] {
def fromBinary(bytes: Array[Byte]) = new String(bytes)
}, new ToBinary[Int] {
def toBinary(t: Int) = Array(t.toByte)
})
def requestHandler(request: String) = 3
val rpcServer = RPC.newRpcServer[String, Int](connection, exchangeName, rpcServerSerializer,
requestHandler _, Some("rpc.routing"), channelParameters = Some(channelParameters))
val rpcClientSerializer = new RpcClientSerializer[String, Int](
new ToBinary[String] {
def toBinary(t: String) = t.getBytes
}, new FromBinary[Int] {
def fromBinary(bytes: Array[Byte]) = bytes.head.toInt
})
val rpcClient = RPC.newRpcClient[String, Int](connection, exchangeName, rpcClientSerializer, Some("rpc.routing"),
channelParameters = Some(channelParameters))
countDown.await(2, TimeUnit.SECONDS) must be(true)
val response = rpcClient.call("some_payload")
response must be(Some(3))
}
}

View file

@ -1,49 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import org.scalatest.matchers.MustMatchers
import org.scalatest.junit.JUnitSuite
import akka.amqp.AMQP
import akka.remote.protocol.RemoteProtocol.AddressProtocol
import org.junit.Test
import akka.amqp.rpc.RPC
import org.multiverse.api.latches.StandardLatch
import java.util.concurrent.TimeUnit
class AMQPRpcProtobufTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
RPC.newProtobufRpcServer(connection, "protoservice", requestHandler)
val protobufClient = RPC.newProtobufRpcClient[AddressProtocol, AddressProtocol](connection, "protoservice")
val request = AddressProtocol.newBuilder.setHostname("testhost").setPort(4321).build
protobufClient.call(request) match {
case Some(response) => assert(response.getHostname == request.getHostname.reverse)
case None => fail("no response")
}
val aSyncLatch = new StandardLatch
protobufClient.callAsync(request) {
case Some(response) => {
assert(response.getHostname == request.getHostname.reverse)
aSyncLatch.open
}
case None => fail("no response")
}
aSyncLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
}
def requestHandler(request: AddressProtocol): AddressProtocol = {
AddressProtocol.newBuilder.setHostname(request.getHostname.reverse).setPort(request.getPort).build
}
}

View file

@ -1,47 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import org.scalatest.matchers.MustMatchers
import org.scalatest.junit.JUnitSuite
import akka.amqp.AMQP
import org.junit.Test
import akka.amqp.rpc.RPC
import org.multiverse.api.latches.StandardLatch
import java.util.concurrent.TimeUnit
class AMQPRpcStringTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
RPC.newStringRpcServer(connection, "stringservice", requestHandler _)
val protobufClient = RPC.newStringRpcClient(connection, "stringservice")
val request = "teststring"
protobufClient.call(request) match {
case Some(response) => assert(response == request.reverse)
case None => fail("no response")
}
val aSyncLatch = new StandardLatch
protobufClient.callAsync(request) {
case Some(response) => {
assert(response == request.reverse)
aSyncLatch.open
}
case None => fail("no response")
}
aSyncLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
}
def requestHandler(request: String): String= {
request.reverse
}
}

View file

@ -1,44 +0,0 @@
package akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import org.scalatest.matchers.MustMatchers
import org.scalatest.junit.JUnitSuite
import akka.amqp.AMQP
import org.junit.Test
import org.multiverse.api.latches.StandardLatch
import java.util.concurrent.TimeUnit
import akka.amqp.rpc.RPC
class AMQPStringProducerConsumerTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
val responseLatch = new StandardLatch
RPC.newStringRpcServer(connection, "stringexchange", requestHandler _)
val request = "somemessage"
def responseHandler(response: String) = {
assert(response == request.reverse)
responseLatch.open
}
AMQP.newStringConsumer(connection, responseHandler _, None, Some("string.reply.key"))
val producer = AMQP.newStringProducer(connection, Some("stringexchange"))
producer.send(request, Some("string.reply.key"))
responseLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
}
def requestHandler(request: String): String= {
println("###### Reverse")
request.reverse
}
}

View file

@ -1,22 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.amqp.test
import akka.amqp.AMQP
object AMQPTest {
def withCleanEndState(action: => Unit) {
try {
try {
action
} finally {
AMQP.shutdownAll
}
} catch {
case e => println(e)
}
}
}

View file

@ -1,34 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.camel;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
* Annotation used by implementations of {@link akka.actor.TypedActor}
* (on method-level) to define consumer endpoints.
*
* @author Martin Krasser
*/
@Retention(RetentionPolicy.RUNTIME)
@Target({ElementType.METHOD})
public @interface consume {
/**
* Consumer endpoint URI
*/
public abstract String value();
/**
* Route definition handler class for customizing route to annotated method.
* The handler class must have a default constructor.
*/
public abstract Class<? extends RouteDefinitionHandler> routeDefinitionHandler()
default RouteDefinitionIdentity.class;
}

View file

@ -1 +0,0 @@
class=akka.camel.component.ActorComponent

View file

@ -1 +0,0 @@
class=akka.camel.component.TypedActorComponent

View file

@ -1,202 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.camel
import java.util.Map
import org.apache.camel.{ProducerTemplate, CamelContext}
import org.apache.camel.impl.DefaultCamelContext
import akka.camel.component.TypedActorComponent
import akka.japi.{Option => JOption}
import akka.util.Logging
/**
* Manages the lifecycle of a CamelContext. Allowed transitions are
* init -> start -> stop -> init -> ... etc.
*
* @author Martin Krasser
*/
trait CamelContextLifecycle extends Logging {
// TODO: enforce correct state transitions
// valid: init -> start -> stop -> init ...
private var _context: Option[CamelContext] = None
private var _template: Option[ProducerTemplate] = None
private var _initialized = false
private var _started = false
/**
* Camel component for accessing typed actors.
*/
private[camel] var typedActorComponent: TypedActorComponent = _
/**
* Registry in which typed actors are TEMPORARILY registered during
* creation of Camel routes to these actors.
*/
private[camel] var typedActorRegistry: Map[String, AnyRef] = _
/**
* Returns <code>Some(CamelContext)</code> (containing the current CamelContext)
* if <code>CamelContextLifecycle</code> has been initialized, otherwise <code>None</code>.
*/
def context: Option[CamelContext] = _context
/**
* Returns <code>Some(ProducerTemplate)</code> (containing the current ProducerTemplate)
* if <code>CamelContextLifecycle</code> has been initialized, otherwise <code>None</code>.
*/
def template: Option[ProducerTemplate] = _template
/**
* Returns <code>Some(CamelContext)</code> (containing the current CamelContext)
* if <code>CamelContextLifecycle</code> has been initialized, otherwise <code>None</code>.
* <p>
* Java API.
*/
def getContext: JOption[CamelContext] = context
/**
* Returns <code>Some(ProducerTemplate)</code> (containing the current ProducerTemplate)
* if <code>CamelContextLifecycle</code> has been initialized, otherwise <code>None</code>.
* <p>
* Java API.
*/
def getTemplate: JOption[ProducerTemplate] = template
/**
* Returns the current <code>CamelContext</code> if this <code>CamelContextLifecycle</code>
* has been initialized, otherwise throws an <code>IllegalStateException</code>.
*/
def mandatoryContext =
if (context.isDefined) context.get
else throw new IllegalStateException("no current CamelContext")
/**
* Returns the current <code>ProducerTemplate</code> if this <code>CamelContextLifecycle</code>
* has been initialized, otherwise throws an <code>IllegalStateException</code>.
*/
def mandatoryTemplate =
if (template.isDefined) template.get
else throw new IllegalStateException("no current ProducerTemplate")
/**
* Returns the current <code>CamelContext</code> if this <code>CamelContextLifecycle</code>
* has been initialized, otherwise throws an <code>IllegalStateException</code>.
* <p>
* Java API.
*/
def getMandatoryContext = mandatoryContext
/**
* Returns the current <code>ProducerTemplate</code> if this <code>CamelContextLifecycle</code>
* has been initialized, otherwise throws an <code>IllegalStateException</code>.
* <p>
* Java API.
*/
def getMandatoryTemplate = mandatoryTemplate
def initialized = _initialized
def started = _started
/**
* Starts the CamelContext and an associated ProducerTemplate.
*/
def start = {
for {
c <- context
t <- template
} {
c.start
t.start
_started = true
log.info("Camel context started")
}
}
/**
* Stops the CamelContext and the associated ProducerTemplate.
*/
def stop = {
for {
t <- template
c <- context
} {
t.stop
c.stop
_started = false
_initialized = false
log.info("Camel context stopped")
}
}
/**
* Initializes this lifecycle object with the a DefaultCamelContext.
*/
def init(): Unit = init(new DefaultCamelContext)
/**
* Initializes this lifecycle object with the given CamelContext. For the passed
* CamelContext, stream-caching is enabled. If applications want to disable stream-
* caching they can do so after this method returned and prior to calling start.
* This method also registers a new TypedActorComponent at the passes CamelContext
* under a name defined by TypedActorComponent.InternalSchema.
*/
def init(context: CamelContext) {
this.typedActorComponent = new TypedActorComponent
this.typedActorRegistry = typedActorComponent.typedActorRegistry
context.setStreamCaching(true)
context.addComponent(TypedActorComponent.InternalSchema, typedActorComponent)
this._context = Some(context)
this._template = Some(context.createProducerTemplate)
_initialized = true
log.info("Camel context initialized")
}
}
/**
* Manages a global CamelContext and an associated ProducerTemplate.
*/
object CamelContextManager extends CamelContextLifecycle {
// -----------------------------------------------------
// The inherited getters aren't statically accessible
// from Java. Therefore, they are redefined here.
// TODO: investigate if this is a Scala bug.
// -----------------------------------------------------
/**
* see CamelContextLifecycle.getContext
* <p>
* Java API.
*/
override def getContext: JOption[CamelContext] = super.getContext
/**
* see CamelContextLifecycle.getTemplate
* <p>
* Java API.
*/
override def getTemplate: JOption[ProducerTemplate] = super.getTemplate
/**
* see CamelContextLifecycle.getMandatoryContext
* <p>
* Java API.
*/
override def getMandatoryContext = super.getMandatoryContext
/**
* see CamelContextLifecycle.getMandatoryTemplate
* <p>
* Java API.
*/
override def getMandatoryTemplate = super.getMandatoryTemplate
}

View file

@ -1,275 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.camel
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import org.apache.camel.CamelContext
import akka.actor.Actor._
import akka.actor.{AspectInitRegistry, ActorRegistry}
import akka.config.Config._
import akka.japi.{SideEffect, Option => JOption}
import akka.util.{Logging, Bootable}
/**
* Publishes (untyped) consumer actors and typed consumer actors via Camel endpoints. Actors
* are published (asynchronously) when they are started and unpublished (asynchronously) when
* they are stopped. The CamelService is notified about actor start- and stop-events by
* registering listeners at ActorRegistry and AspectInitRegistry.
*
* @author Martin Krasser
*/
trait CamelService extends Bootable with Logging {
private[camel] val consumerPublisher = actorOf[ConsumerPublisher]
private[camel] val publishRequestor = actorOf[PublishRequestor]
private val serviceEnabled = config.getList("akka.enabled-modules").exists(_ == "camel")
/**
* Starts this CamelService unless <code>akka.camel.service</code> is set to <code>false</code>.
*/
abstract override def onLoad = {
if (serviceEnabled) registerPublishRequestor
super.onLoad
if (serviceEnabled) start
}
/**
* Stops this CamelService unless <code>akka.camel.service</code> is set to <code>false</code>.
*/
abstract override def onUnload = {
if (serviceEnabled) stop
super.onUnload
}
@deprecated("use start() instead")
def load = start
@deprecated("use stop() instead")
def unload = stop
/**
* Starts this CamelService. Any started actor that is a consumer actor will be (asynchronously)
* published as Camel endpoint. Consumer actors that are started after this method returned will
* be published as well. Actor publishing is done asynchronously. A started (loaded) CamelService
* also publishes <code>@consume</code> annotated methods of typed actors that have been created
* with <code>TypedActor.newInstance(..)</code> (and <code>TypedActor.newRemoteInstance(..)</code>
* on a remote node).
*/
def start: CamelService = {
if (!publishRequestorRegistered) registerPublishRequestor
// Only init and start if not already done by application
if (!CamelContextManager.initialized) CamelContextManager.init
if (!CamelContextManager.started) CamelContextManager.start
// start actor that exposes consumer actors and typed actors via Camel endpoints
consumerPublisher.start
// init publishRequestor so that buffered and future events are delivered to consumerPublisher
publishRequestor ! PublishRequestorInit(consumerPublisher)
// Register this instance as current CamelService and return it
CamelServiceManager.register(this)
CamelServiceManager.mandatoryService
}
/**
* Stops this CamelService. All published consumer actors and typed consumer actor methods will be
* unpublished asynchronously.
*/
def stop = {
// Unregister this instance as current CamelService
CamelServiceManager.unregister(this)
// Remove related listeners from registry
unregisterPublishRequestor
// Stop related services
consumerPublisher.stop
CamelContextManager.stop
}
/**
* Waits for an expected number (<code>count</code>) of endpoints to be activated
* during execution of <code>f</code>. The wait-timeout is by default 10 seconds.
* Other timeout values can be set via the <code>timeout</code> and <code>timeUnit</code>
* parameters.
*/
def awaitEndpointActivation(count: Int, timeout: Long = 10, timeUnit: TimeUnit = TimeUnit.SECONDS)(f: => Unit): Boolean = {
val activation = expectEndpointActivationCount(count)
f; activation.await(timeout, timeUnit)
}
/**
* Waits for an expected number (<code>count</code>) of endpoints to be de-activated
* during execution of <code>f</code>. The wait-timeout is by default 10 seconds.
* Other timeout values can be set via the <code>timeout</code> and <code>timeUnit</code>
* parameters.
*/
def awaitEndpointDeactivation(count: Int, timeout: Long = 10, timeUnit: TimeUnit = TimeUnit.SECONDS)(f: => Unit): Boolean = {
val activation = expectEndpointDeactivationCount(count)
f; activation.await(timeout, timeUnit)
}
/**
* Waits for an expected number (<code>count</code>) of endpoints to be activated
* during execution of <code>p</code>. The wait timeout is 10 seconds.
* <p>
* Java API
*/
def awaitEndpointActivation(count: Int, p: SideEffect): Boolean = {
awaitEndpointActivation(count, 10, TimeUnit.SECONDS, p)
}
/**
* Waits for an expected number (<code>count</code>) of endpoints to be activated
* during execution of <code>p</code>. Timeout values can be set via the
* <code>timeout</code> and <code>timeUnit</code> parameters.
* <p>
* Java API
*/
def awaitEndpointActivation(count: Int, timeout: Long, timeUnit: TimeUnit, p: SideEffect): Boolean = {
awaitEndpointActivation(count, timeout, timeUnit) { p.apply }
}
/**
* Waits for an expected number (<code>count</code>) of endpoints to be de-activated
* during execution of <code>p</code>. The wait timeout is 10 seconds.
* <p>
* Java API
*/
def awaitEndpointDeactivation(count: Int, p: SideEffect): Boolean = {
awaitEndpointDeactivation(count, 10, TimeUnit.SECONDS, p)
}
/**
* Waits for an expected number (<code>count</code>) of endpoints to be de-activated
* during execution of <code>p</code>. Timeout values can be set via the
* <code>timeout</code> and <code>timeUnit</code> parameters.
* <p>
* Java API
*/
def awaitEndpointDeactivation(count: Int, timeout: Long, timeUnit: TimeUnit, p: SideEffect): Boolean = {
awaitEndpointDeactivation(count, timeout, timeUnit) { p.apply }
}
/**
* Sets an expectation on the number of upcoming endpoint activations and returns
* a CountDownLatch that can be used to wait for the activations to occur. Endpoint
* activations that occurred in the past are not considered.
*/
private def expectEndpointActivationCount(count: Int): CountDownLatch =
(consumerPublisher !! SetExpectedRegistrationCount(count)).as[CountDownLatch].get
/**
* Sets an expectation on the number of upcoming endpoint de-activations and returns
* a CountDownLatch that can be used to wait for the de-activations to occur. Endpoint
* de-activations that occurred in the past are not considered.
*/
private def expectEndpointDeactivationCount(count: Int): CountDownLatch =
(consumerPublisher !! SetExpectedUnregistrationCount(count)).as[CountDownLatch].get
private[camel] def publishRequestorRegistered: Boolean = {
ActorRegistry.hasListener(publishRequestor) ||
AspectInitRegistry.hasListener(publishRequestor)
}
private[camel] def registerPublishRequestor: Unit = {
ActorRegistry.addListener(publishRequestor)
AspectInitRegistry.addListener(publishRequestor)
}
private[camel] def unregisterPublishRequestor: Unit = {
ActorRegistry.removeListener(publishRequestor)
AspectInitRegistry.removeListener(publishRequestor)
}
}
/**
* Manages a global CamelService (the 'current' CamelService).
*
* @author Martin Krasser
*/
object CamelServiceManager {
/**
* The current (optional) CamelService. Is defined when a CamelService has been started.
*/
private var _current: Option[CamelService] = None
/**
* Starts a new CamelService and makes it the current CamelService.
*
* @see CamelService#start
* @see CamelService#onLoad
*/
def startCamelService = CamelServiceFactory.createCamelService.start
/**
* Stops the current CamelService.
*
* @see CamelService#stop
* @see CamelService#onUnload
*/
def stopCamelService = for (s <- service) s.stop
/**
* Returns <code>Some(CamelService)</code> if this <code>CamelService</code>
* has been started, <code>None</code> otherwise.
*/
def service = _current
/**
* Returns the current <code>CamelService</code> if <code>CamelService</code>
* has been started, otherwise throws an <code>IllegalStateException</code>.
* <p>
* Java API
*/
def getService: JOption[CamelService] = CamelServiceManager.service
/**
* Returns <code>Some(CamelService)</code> (containing the current CamelService)
* if this <code>CamelService</code>has been started, <code>None</code> otherwise.
*/
def mandatoryService =
if (_current.isDefined) _current.get
else throw new IllegalStateException("co current CamelService")
/**
* Returns <code>Some(CamelService)</code> (containing the current CamelService)
* if this <code>CamelService</code>has been started, <code>None</code> otherwise.
* <p>
* Java API
*/
def getMandatoryService = mandatoryService
private[camel] def register(service: CamelService) =
if (_current.isDefined) throw new IllegalStateException("current CamelService already registered")
else _current = Some(service)
private[camel] def unregister(service: CamelService) =
if (_current == Some(service)) _current = None
else throw new IllegalStateException("only current CamelService can be unregistered")
}
/**
* @author Martin Krasser
*/
object CamelServiceFactory {
/**
* Creates a new CamelService instance.
*/
def createCamelService: CamelService = new CamelService { }
/**
* Creates a new CamelService instance and initializes it with the given CamelContext.
*/
def createCamelService(camelContext: CamelContext): CamelService = {
CamelContextManager.init(camelContext)
createCamelService
}
}

View file

@ -1,145 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.camel
import java.net.InetSocketAddress
import org.apache.camel.{Exchange, Processor}
import org.apache.camel.model.{RouteDefinition, ProcessorDefinition}
import akka.actor._
import akka.japi.{Function => JFunction}
/**
* Mixed in by Actor implementations that consume message from Camel endpoints.
*
* @author Martin Krasser
*/
trait Consumer { self: Actor =>
import RouteDefinitionHandler._
/**
* The default route definition handler is the identity function
*/
private[camel] var routeDefinitionHandler: RouteDefinitionHandler = identity
/**
* Returns the Camel endpoint URI to consume messages from.
*/
def endpointUri: String
/**
* Determines whether two-way communications between an endpoint and this consumer actor
* should be done in blocking or non-blocking mode (default is non-blocking). This method
* doesn't have any effect on one-way communications (they'll never block).
*/
def blocking = false
/**
* Sets the route definition handler for creating a custom route to this consumer instance.
*/
def onRouteDefinition(h: RouteDefinition => ProcessorDefinition[_]): Unit = onRouteDefinition(from(h))
/**
* Sets the route definition handler for creating a custom route to this consumer instance.
* <p>
* Java API.
*/
def onRouteDefinition(h: RouteDefinitionHandler): Unit = routeDefinitionHandler = h
}
/**
* Java-friendly Consumer.
*
* @see UntypedConsumerActor
* @see RemoteUntypedConsumerActor
*
* @author Martin Krasser
*/
trait UntypedConsumer extends Consumer { self: UntypedActor =>
final override def endpointUri = getEndpointUri
final override def blocking = isBlocking
/**
* Returns the Camel endpoint URI to consume messages from.
*/
def getEndpointUri(): String
/**
* Determines whether two-way communications between an endpoint and this consumer actor
* should be done in blocking or non-blocking mode (default is non-blocking). This method
* doesn't have any effect on one-way communications (they'll never block).
*/
def isBlocking() = super.blocking
}
/**
* Subclass this abstract class to create an MDB-style untyped consumer actor. This
* class is meant to be used from Java.
*/
abstract class UntypedConsumerActor extends UntypedActor with UntypedConsumer
/**
* Subclass this abstract class to create an MDB-style remote untyped consumer
* actor. This class is meant to be used from Java.
*/
abstract class RemoteUntypedConsumerActor(address: InetSocketAddress) extends RemoteUntypedActor(address) with UntypedConsumer {
def this(host: String, port: Int) = this(new InetSocketAddress(host, port))
}
/**
* A callback handler for route definitions to consumer actors.
*
* @author Martin Krasser
*/
trait RouteDefinitionHandler {
def onRouteDefinition(rd: RouteDefinition): ProcessorDefinition[_]
}
/**
* The identity route definition handler.
*
* @author Martin Krasser
*
*/
class RouteDefinitionIdentity extends RouteDefinitionHandler {
def onRouteDefinition(rd: RouteDefinition) = rd
}
/**
* @author Martin Krasser
*/
object RouteDefinitionHandler {
/**
* Returns the identity route definition handler
*/
val identity = new RouteDefinitionIdentity
/**
* Created a route definition handler from the given function.
*/
def from(f: RouteDefinition => ProcessorDefinition[_]) = new RouteDefinitionHandler {
def onRouteDefinition(rd: RouteDefinition) = f(rd)
}
}
/**
* @author Martin Krasser
*/
private[camel] object Consumer {
/**
* Applies a function <code>f</code> to <code>actorRef</code> if <code>actorRef</code>
* references a consumer actor. A valid reference to a consumer actor is a local actor
* reference with a target actor that implements the <code>Consumer</code> trait. The
* target <code>Consumer</code> object is passed as argument to <code>f</code>. This
* method returns <code>None</code> if <code>actorRef</code> is not a valid reference
* to a consumer actor, <code>Some</code> consumer actor otherwise.
*/
def forConsumer[T](actorRef: ActorRef)(f: Consumer => T): Option[T] = {
if (!actorRef.actor.isInstanceOf[Consumer]) None
else if (actorRef.remoteAddress.isDefined) None
else Some(f(actorRef.actor.asInstanceOf[Consumer]))
}
}

View file

@ -1,351 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.camel
import collection.mutable.ListBuffer
import java.io.InputStream
import java.lang.reflect.Method
import java.util.concurrent.CountDownLatch
import org.apache.camel.builder.RouteBuilder
import org.apache.camel.model.{ProcessorDefinition, RouteDefinition}
import akka.actor._
import akka.camel.component.TypedActorComponent
import akka.util.Logging
/**
* @author Martin Krasser
*/
private[camel] object ConsumerPublisher extends Logging {
/**
* Creates a route to the registered consumer actor.
*/
def handleConsumerActorRegistered(event: ConsumerActorRegistered) {
CamelContextManager.mandatoryContext.addRoutes(new ConsumerActorRouteBuilder(event))
log.info("published actor %s at endpoint %s" format (event.actorRef, event.endpointUri))
}
/**
* Stops the route to the already un-registered consumer actor.
*/
def handleConsumerActorUnregistered(event: ConsumerActorUnregistered) {
CamelContextManager.mandatoryContext.stopRoute(event.uuid)
log.info("unpublished actor %s from endpoint %s" format (event.actorRef, event.endpointUri))
}
/**
* Creates a route to an typed actor method.
*/
def handleConsumerMethodRegistered(event: ConsumerMethodRegistered) {
CamelContextManager.typedActorRegistry.put(event.methodUuid, event.typedActor)
CamelContextManager.mandatoryContext.addRoutes(new ConsumerMethodRouteBuilder(event))
log.info("published method %s of %s at endpoint %s" format (event.methodName, event.typedActor, event.endpointUri))
}
/**
* Stops the route to the already un-registered consumer actor method.
*/
def handleConsumerMethodUnregistered(event: ConsumerMethodUnregistered) {
CamelContextManager.typedActorRegistry.remove(event.methodUuid)
CamelContextManager.mandatoryContext.stopRoute(event.methodUuid)
log.info("unpublished method %s of %s from endpoint %s" format (event.methodName, event.typedActor, event.endpointUri))
}
}
/**
* Actor that publishes consumer actors and typed actor methods at Camel endpoints.
* The Camel context used for publishing is obtained via CamelContextManager.context.
* This actor accepts messages of type
* akka.camel.ConsumerActorRegistered,
* akka.camel.ConsumerActorUnregistered,
* akka.camel.ConsumerMethodRegistered and
* akka.camel.ConsumerMethodUnregistered.
*
* @author Martin Krasser
*/
private[camel] class ConsumerPublisher extends Actor {
import ConsumerPublisher._
@volatile private var registrationLatch = new CountDownLatch(0)
@volatile private var unregistrationLatch = new CountDownLatch(0)
protected def receive = {
case r: ConsumerActorRegistered => {
handleConsumerActorRegistered(r)
registrationLatch.countDown
}
case u: ConsumerActorUnregistered => {
handleConsumerActorUnregistered(u)
unregistrationLatch.countDown
}
case mr: ConsumerMethodRegistered => {
handleConsumerMethodRegistered(mr)
registrationLatch.countDown
}
case mu: ConsumerMethodUnregistered => {
handleConsumerMethodUnregistered(mu)
unregistrationLatch.countDown
}
case SetExpectedRegistrationCount(num) => {
registrationLatch = new CountDownLatch(num)
self.reply(registrationLatch)
}
case SetExpectedUnregistrationCount(num) => {
unregistrationLatch = new CountDownLatch(num)
self.reply(unregistrationLatch)
}
case _ => { /* ignore */}
}
}
private[camel] case class SetExpectedRegistrationCount(num: Int)
private[camel] case class SetExpectedUnregistrationCount(num: Int)
/**
* Abstract route to a target which is either an actor or an typed actor method.
*
* @param endpointUri endpoint URI of the consumer actor or typed actor method.
* @param id actor identifier or typed actor identifier (registry key).
*
* @author Martin Krasser
*/
private[camel] abstract class ConsumerRouteBuilder(endpointUri: String, id: String) extends RouteBuilder {
// TODO: make conversions configurable
private val bodyConversions = Map(
"file" -> classOf[InputStream]
)
def configure = {
val schema = endpointUri take endpointUri.indexOf(":") // e.g. "http" from "http://whatever/..."
val cnvopt = bodyConversions.get(schema)
onRouteDefinition(startRouteDefinition(cnvopt)).to(targetUri)
}
protected def routeDefinitionHandler: RouteDefinitionHandler
protected def targetUri: String
private def onRouteDefinition(rd: RouteDefinition) = routeDefinitionHandler.onRouteDefinition(rd)
private def startRouteDefinition(bodyConversion: Option[Class[_]]): RouteDefinition = bodyConversion match {
case Some(clazz) => from(endpointUri).routeId(id).convertBodyTo(clazz)
case None => from(endpointUri).routeId(id)
}
}
/**
* Defines the route to a (untyped) consumer actor.
*
* @author Martin Krasser
*/
private[camel] class ConsumerActorRouteBuilder(event: ConsumerActorRegistered) extends ConsumerRouteBuilder(event.endpointUri, event.uuid) {
protected def routeDefinitionHandler: RouteDefinitionHandler = event.routeDefinitionHandler
protected def targetUri = "actor:uuid:%s?blocking=%s" format (event.uuid, event.blocking)
}
/**
* Defines the route to a typed actor method.
*
* @author Martin Krasser
*/
private[camel] class ConsumerMethodRouteBuilder(event: ConsumerMethodRegistered) extends ConsumerRouteBuilder(event.endpointUri, event.methodUuid) {
protected def routeDefinitionHandler: RouteDefinitionHandler = event.routeDefinitionHandler
protected def targetUri = "%s:%s?method=%s" format (TypedActorComponent.InternalSchema, event.methodUuid, event.methodName)
}
/**
* A registration listener that triggers publication of consumer actors and typed actor
* methods as well as un-publication of consumer actors and typed actor methods. This actor
* needs to be initialized with a <code>PublishRequestorInit</code> command message for
* obtaining a reference to a <code>publisher</code> actor. Before initialization it buffers
* all outbound messages and delivers them to the <code>publisher</code> when receiving a
* <code>PublishRequestorInit</code> message. After initialization, outbound messages are
* delivered directly without buffering.
*
* @see PublishRequestorInit
*
* @author Martin Krasser
*/
private[camel] class PublishRequestor extends Actor {
private val events = ListBuffer[ConsumerEvent]()
private var publisher: Option[ActorRef] = None
protected def receive = {
case ActorRegistered(actor) =>
for (event <- ConsumerActorRegistered.forConsumer(actor)) deliverCurrentEvent(event)
case ActorUnregistered(actor) =>
for (event <- ConsumerActorUnregistered.forConsumer(actor)) deliverCurrentEvent(event)
case AspectInitRegistered(proxy, init) =>
for (event <- ConsumerMethodRegistered.forConsumer(proxy, init)) deliverCurrentEvent(event)
case AspectInitUnregistered(proxy, init) =>
for (event <- ConsumerMethodUnregistered.forConsumer(proxy, init)) deliverCurrentEvent(event)
case PublishRequestorInit(pub) => {
publisher = Some(pub)
deliverBufferedEvents
}
case _ => { /* ignore */ }
}
private def deliverCurrentEvent(event: ConsumerEvent) = {
publisher match {
case Some(pub) => pub ! event
case None => events += event
}
}
private def deliverBufferedEvents = {
for (event <- events) deliverCurrentEvent(event)
events.clear
}
}
/**
* Command message to initialize a PublishRequestor to use <code>consumerPublisher</code>
* for publishing actors or typed actor methods.
*/
private[camel] case class PublishRequestorInit(consumerPublisher: ActorRef)
/**
* A consumer (un)registration event.
*/
private[camel] sealed trait ConsumerEvent
/**
* A consumer actor (un)registration event.
*/
private[camel] trait ConsumerActorEvent extends ConsumerEvent {
val actorRef: ActorRef
val actor: Consumer
val uuid = actorRef.uuid.toString
val endpointUri = actor.endpointUri
val blocking = actor.blocking
val routeDefinitionHandler = actor.routeDefinitionHandler
}
/**
* A consumer method (un)registration event.
*/
private[camel] trait ConsumerMethodEvent extends ConsumerEvent {
val typedActor: AnyRef
val init: AspectInit
val method: Method
val uuid = init.actorRef.uuid.toString
val methodName = method.getName
val methodUuid = "%s_%s" format (uuid, methodName)
lazy val routeDefinitionHandler = consumeAnnotation.routeDefinitionHandler.newInstance
lazy val consumeAnnotation = method.getAnnotation(classOf[consume])
lazy val endpointUri = consumeAnnotation.value
}
/**
* Event indicating that a consumer actor has been registered at the actor registry.
*/
private[camel] case class ConsumerActorRegistered(actorRef: ActorRef, actor: Consumer) extends ConsumerActorEvent
/**
* Event indicating that a consumer actor has been unregistered from the actor registry.
*/
private[camel] case class ConsumerActorUnregistered(actorRef: ActorRef, actor: Consumer) extends ConsumerActorEvent
/**
* Event indicating that an typed actor proxy has been created for a typed actor. For each <code>@consume</code>
* annotated typed actor method a separate instance of this class is created.
*/
private[camel] case class ConsumerMethodRegistered(typedActor: AnyRef, init: AspectInit, method: Method) extends ConsumerMethodEvent
/**
* Event indicating that an typed actor has been stopped. For each <code>@consume</code>
* annotated typed object method a separate instance of this class is created.
*/
private[camel] case class ConsumerMethodUnregistered(typedActor: AnyRef, init: AspectInit, method: Method) extends ConsumerMethodEvent
/**
* @author Martin Krasser
*/
private[camel] object ConsumerActorRegistered {
/**
* Creates an ConsumerActorRegistered event message for a consumer actor or None if
* <code>actorRef</code> is not a consumer actor.
*/
def forConsumer(actorRef: ActorRef): Option[ConsumerActorRegistered] = {
Consumer.forConsumer[ConsumerActorRegistered](actorRef) {
actor => ConsumerActorRegistered(actorRef, actor)
}
}
}
/**
* @author Martin Krasser
*/
private[camel] object ConsumerActorUnregistered {
/**
* Creates an ConsumerActorUnregistered event message for a consumer actor or None if
* <code>actorRef</code> is not a consumer actor.
*/
def forConsumer(actorRef: ActorRef): Option[ConsumerActorUnregistered] = {
Consumer.forConsumer[ConsumerActorUnregistered](actorRef) {
actor => ConsumerActorUnregistered(actorRef, actor)
}
}
}
/**
* @author Martin Krasser
*/
private[camel] object ConsumerMethod {
/**
* Applies a function <code>f</code> to each consumer method of <code>TypedActor</code> and
* returns the function results as a list. A consumer method is one that is annotated with
* <code>@consume</code>. If <code>typedActor</code> is a proxy for a remote typed actor
* <code>f</code> is never called and <code>Nil</code> is returned.
*/
def forConsumer[T](typedActor: AnyRef, init: AspectInit)(f: Method => T): List[T] = {
if (init.remoteAddress.isDefined) Nil // let remote node publish typed actor methods on endpoints
else {
// TODO: support consumer annotation inheritance
// - visit overridden methods in superclasses
// - visit implemented method declarations in interfaces
val intfClass = typedActor.getClass
val implClass = init.targetInstance.getClass
(for (m <- intfClass.getMethods.toList; if (m.isAnnotationPresent(classOf[consume]))) yield f(m)) ++
(for (m <- implClass.getMethods.toList; if (m.isAnnotationPresent(classOf[consume]))) yield f(m))
}
}
}
/**
* @author Martin Krasser
*/
private[camel] object ConsumerMethodRegistered {
/**
* Creates a list of ConsumerMethodRegistered event messages for a typed actor or an empty
* list if the typed actor is a proxy for a remote typed actor or the typed actor doesn't
* have any <code>@consume</code> annotated methods.
*/
def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodRegistered] = {
ConsumerMethod.forConsumer(typedActor, init) {
m => ConsumerMethodRegistered(typedActor, init, m)
}
}
}
/**
* @author Martin Krasser
*/
private[camel] object ConsumerMethodUnregistered {
/**
* Creates a list of ConsumerMethodUnregistered event messages for a typed actor or an empty
* list if the typed actor is a proxy for a remote typed actor or the typed actor doesn't
* have any <code>@consume</code> annotated methods.
*/
def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodUnregistered] = {
ConsumerMethod.forConsumer(typedActor, init) {
m => ConsumerMethodUnregistered(typedActor, init, m)
}
}
}

View file

@ -1,380 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.camel
import java.util.{Map => JMap, Set => JSet}
import scala.collection.JavaConversions._
import org.apache.camel.{Exchange, Message => CamelMessage}
import org.apache.camel.util.ExchangeHelper
import akka.japi.{Function => JFunction}
/**
* An immutable representation of a Camel message.
*
* @author Martin Krasser
*/
case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
/**
* Creates a Message with given body and empty headers map.
*/
def this(body: Any) = this(body, Map.empty[String, Any])
/**
* Creates a Message with given body and headers map. A copy of the headers map is made.
* <p>
* Java API
*/
def this(body: Any, headers: JMap[String, Any]) = this(body, headers.toMap)
/**
* Returns the body of the message converted to the type <code>T</code>. Conversion is done
* using Camel's type converter. The type converter is obtained from the CamelContext managed
* by CamelContextManager. Applications have to ensure proper initialization of
* CamelContextManager.
*
* @see CamelContextManager.
*/
def bodyAs[T](implicit m: Manifest[T]): T = getBodyAs(m.erasure.asInstanceOf[Class[T]])
/**
* Returns the body of the message converted to the type as given by the <code>clazz</code>
* parameter. Conversion is done using Camel's type converter. The type converter is obtained
* from the CamelContext managed by CamelContextManager. Applications have to ensure proper
* initialization of CamelContextManager.
* <p>
* Java API
*
* @see CamelContextManager.
*/
def getBodyAs[T](clazz: Class[T]): T =
CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](clazz, body)
/**
* Returns those headers from this message whose name is contained in <code>names</code>.
*/
def headers(names: Set[String]): Map[String, Any] = headers.filter(names contains _._1)
/**
* Returns those headers from this message whose name is contained in <code>names</code>.
* The returned headers map is backed up by an immutable headers map. Any attempt to modify
* the returned map will throw an exception.
* <p>
* Java API
*/
def getHeaders(names: JSet[String]): JMap[String, Any] = headers.filter(names contains _._1)
/**
* Returns all headers from this message. The returned headers map is backed up by this
* message's immutable headers map. Any attempt to modify the returned map will throw an
* exception.
* <p>
* Java API
*/
def getHeaders: JMap[String, Any] = headers
/**
* Returns the header with given <code>name</code>. Throws <code>NoSuchElementException</code>
* if the header doesn't exist.
*/
def header(name: String): Any = headers(name)
/**
* Returns the header with given <code>name</code>. Throws <code>NoSuchElementException</code>
* if the header doesn't exist.
* <p>
* Java API
*/
def getHeader(name: String): Any = header(name)
/**
* Returns the header with given <code>name</code> converted to type <code>T</code>. Throws
* <code>NoSuchElementException</code> if the header doesn't exist.
*/
def headerAs[T](name: String)(implicit m: Manifest[T]): T =
getHeaderAs(name, m.erasure.asInstanceOf[Class[T]])
/**
* Returns the header with given <code>name</code> converted to type as given by the <code>clazz</code>
* parameter. Throws <code>NoSuchElementException</code> if the header doesn't exist.
* <p>
* Java API
*/
def getHeaderAs[T](name: String, clazz: Class[T]): T =
CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](clazz, header(name))
/**
* Creates a Message with a transformed body using a <code>transformer</code> function.
*/
def transformBody[A](transformer: A => Any): Message = setBody(transformer(body.asInstanceOf[A]))
/**
* Creates a Message with a transformed body using a <code>transformer</code> function.
* <p>
* Java API
*/
def transformBody[A](transformer: JFunction[A, Any]): Message = setBody(transformer(body.asInstanceOf[A]))
/**
* Creates a Message with current <code>body</code> converted to type <code>T</code>.
*/
def setBodyAs[T](implicit m: Manifest[T]): Message = setBodyAs(m.erasure.asInstanceOf[Class[T]])
/**
* Creates a Message with current <code>body</code> converted to type <code>clazz</code>.
* <p>
* Java API
*/
def setBodyAs[T](clazz: Class[T]): Message = setBody(getBodyAs(clazz))
/**
* Creates a Message with a given <code>body</code>.
*/
def setBody(body: Any) = new Message(body, this.headers)
/**
* Creates a new Message with given <code>headers</code>.
*/
def setHeaders(headers: Map[String, Any]): Message = copy(this.body, headers)
/**
* Creates a new Message with given <code>headers</code>. A copy of the headers map is made.
* <p>
* Java API
*/
def setHeaders(headers: JMap[String, Any]): Message = setHeaders(headers.toMap)
/**
* Creates a new Message with given <code>headers</code> added to the current headers.
*/
def addHeaders(headers: Map[String, Any]): Message = copy(this.body, this.headers ++ headers)
/**
* Creates a new Message with given <code>headers</code> added to the current headers.
* A copy of the headers map is made.
* <p>
* Java API
*/
def addHeaders(headers: JMap[String, Any]): Message = addHeaders(headers.toMap)
/**
* Creates a new Message with the given <code>header</code> added to the current headers.
*/
def addHeader(header: (String, Any)): Message = copy(this.body, this.headers + header)
/**
* Creates a new Message with the given header, represented by <code>name</code> and
* <code>value</code> added to the existing headers.
* <p>
* Java API
*/
def addHeader(name: String, value: Any): Message = addHeader((name, value))
/**
* Creates a new Message where the header with given <code>headerName</code> is removed from
* the existing headers.
*/
def removeHeader(headerName: String) = copy(this.body, this.headers - headerName)
}
/**
* Companion object of Message class.
*
* @author Martin Krasser
*/
object Message {
/**
* Message header to correlate request with response messages. Applications that send
* messages to a Producer actor may want to set this header on the request message
* so that it can be correlated with an asynchronous response. Messages send to Consumer
* actors have this header already set.
*/
val MessageExchangeId = "MessageExchangeId".intern
/**
* Creates a new Message with <code>body</code> as message body and an empty header map.
*/
//def apply(body: Any) = new Message(body)
/**
* Creates a canonical form of the given message <code>msg</code>. If <code>msg</code> of type
* Message then <code>msg</code> is returned, otherwise <code>msg</code> is set as body of a
* newly created Message object.
*/
def canonicalize(msg: Any) = msg match {
case mobj: Message => mobj
case body => new Message(body)
}
}
/**
* An immutable representation of a failed Camel exchange. It contains the failure cause
* obtained from Exchange.getException and the headers from either the Exchange.getIn
* message or Exchange.getOut message, depending on the exchange pattern.
*
* @author Martin Krasser
*/
case class Failure(val cause: Exception, val headers: Map[String, Any] = Map.empty) {
/**
* Creates a Failure with cause body and empty headers map.
*/
def this(cause: Exception) = this(cause, Map.empty[String, Any])
/**
* Creates a Failure with given cause and headers map. A copy of the headers map is made.
* <p>
* Java API
*/
def this(cause: Exception, headers: JMap[String, Any]) = this(cause, headers.toMap)
/**
* Returns the cause of this Failure.
* <p>
* Java API.
*/
def getCause = cause
/**
* Returns all headers from this failure message. The returned headers map is backed up by
* this message's immutable headers map. Any attempt to modify the returned map will throw
* an exception.
* <p>
* Java API
*/
def getHeaders: JMap[String, Any] = headers
}
/**
* Adapter for converting an org.apache.camel.Exchange to and from Message and Failure objects.
*
* @author Martin Krasser
*/
class CamelExchangeAdapter(exchange: Exchange) {
import CamelMessageConversion.toMessageAdapter
/**
* Sets Exchange.getIn from the given Message object.
*/
def fromRequestMessage(msg: Message): Exchange = { requestMessage.fromMessage(msg); exchange }
/**
* Depending on the exchange pattern, sets Exchange.getIn or Exchange.getOut from the given
* Message object. If the exchange is out-capable then the Exchange.getOut is set, otherwise
* Exchange.getIn.
*/
def fromResponseMessage(msg: Message): Exchange = { responseMessage.fromMessage(msg); exchange }
/**
* Sets Exchange.getException from the given Failure message. Headers of the Failure message
* are ignored.
*/
def fromFailureMessage(msg: Failure): Exchange = { exchange.setException(msg.cause); exchange }
/**
* Creates a Message object from Exchange.getIn.
*/
def toRequestMessage: Message = toRequestMessage(Map.empty)
/**
* Depending on the exchange pattern, creates a Message object from Exchange.getIn or Exchange.getOut.
* If the exchange is out-capable then the Exchange.getOut is set, otherwise Exchange.getIn.
*/
def toResponseMessage: Message = toResponseMessage(Map.empty)
/**
* Creates a Failure object from the adapted Exchange.
*
* @see Failure
*/
def toFailureMessage: Failure = toFailureMessage(Map.empty)
/**
* Creates a Message object from Exchange.getIn.
*
* @param headers additional headers to set on the created Message in addition to those
* in the Camel message.
*/
def toRequestMessage(headers: Map[String, Any]): Message = requestMessage.toMessage(headers)
/**
* Depending on the exchange pattern, creates a Message object from Exchange.getIn or Exchange.getOut.
* If the exchange is out-capable then the Exchange.getOut is set, otherwise Exchange.getIn.
*
* @param headers additional headers to set on the created Message in addition to those
* in the Camel message.
*/
def toResponseMessage(headers: Map[String, Any]): Message = responseMessage.toMessage(headers)
/**
* Creates a Failure object from the adapted Exchange.
*
* @param headers additional headers to set on the created Message in addition to those
* in the Camel message.
*
* @see Failure
*/
def toFailureMessage(headers: Map[String, Any]): Failure =
Failure(exchange.getException, headers ++ responseMessage.toMessage.headers)
private def requestMessage = exchange.getIn
private def responseMessage = ExchangeHelper.getResultMessage(exchange)
}
/**
* Adapter for converting an org.apache.camel.Message to and from Message objects.
*
* @author Martin Krasser
*/
class CamelMessageAdapter(val cm: CamelMessage) {
/**
* Set the adapted Camel message from the given Message object.
*/
def fromMessage(m: Message): CamelMessage = {
cm.setBody(m.body)
for (h <- m.headers) cm.getHeaders.put(h._1, h._2.asInstanceOf[AnyRef])
cm
}
/**
* Creates a new Message object from the adapted Camel message.
*/
def toMessage: Message = toMessage(Map.empty)
/**
* Creates a new Message object from the adapted Camel message.
*
* @param headers additional headers to set on the created Message in addition to those
* in the Camel message.
*/
def toMessage(headers: Map[String, Any]): Message = Message(cm.getBody, cmHeaders(headers, cm))
private def cmHeaders(headers: Map[String, Any], cm: CamelMessage) = headers ++ cm.getHeaders
}
/**
* Defines conversion methods to CamelExchangeAdapter and CamelMessageAdapter.
* Imported by applications that implicitly want to use conversion methods of
* CamelExchangeAdapter and CamelMessageAdapter.
*/
object CamelMessageConversion {
/**
* Creates an CamelExchangeAdapter for the given Camel exchange.
*/
implicit def toExchangeAdapter(ce: Exchange): CamelExchangeAdapter =
new CamelExchangeAdapter(ce)
/**
* Creates an CamelMessageAdapter for the given Camel message.
*/
implicit def toMessageAdapter(cm: CamelMessage): CamelMessageAdapter =
new CamelMessageAdapter(cm)
}

View file

@ -1,256 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.camel
import CamelMessageConversion.toExchangeAdapter
import org.apache.camel._
import org.apache.camel.processor.SendProcessor
import akka.actor.{Actor, ActorRef, UntypedActor}
/**
* Support trait for producing messages to Camel endpoints.
*
* @author Martin Krasser
*/
trait ProducerSupport { this: Actor =>
/**
* Message headers to copy by default from request message to response-message.
*/
private val headersToCopyDefault = Set(Message.MessageExchangeId)
/**
* <code>Endpoint</code> object resolved from the current CamelContext with
* <code>endpointUri</code>.
*/
private lazy val endpoint = CamelContextManager.mandatoryContext.getEndpoint(endpointUri)
/**
* <code>SendProcessor</code> for producing messages to <code>endpoint</code>.
*/
private lazy val processor = createSendProcessor
/**
* If set to false (default), this producer expects a response message from the Camel endpoint.
* If set to true, this producer initiates an in-only message exchange with the Camel endpoint
* (fire and forget).
*/
def oneway: Boolean = false
/**
* Returns the Camel endpoint URI to produce messages to.
*/
def endpointUri: String
/**
* Returns the names of message headers to copy from a request message to a response message.
* By default only the Message.MessageExchangeId is copied. Applications may override this to
* define an application-specific set of message headers to copy.
*/
def headersToCopy: Set[String] = headersToCopyDefault
/**
* Default implementation of <code>Actor.postStop</code> for freeing resources needed
* to actually send messages to <code>endpointUri</code>.
*/
override def postStop {
processor.stop
}
/**
* Initiates a message exchange of given <code>pattern</code> with the endpoint specified by
* <code>endpointUri</code>. The in-message of the initiated exchange is the canonical form
* of <code>msg</code>. After sending the in-message, the processing result (response) is passed
* as argument to <code>receiveAfterProduce</code>. If the response is received synchronously from
* the endpoint then <code>receiveAfterProduce</code> is called synchronously as well. If the
* response is received asynchronously, the <code>receiveAfterProduce</code> is called
* asynchronously. This is done by wrapping the response, adding it to this producers
* mailbox, unwrapping it and calling <code>receiveAfterProduce</code>. The original
* sender and senderFuture are thereby preserved.
*
* @see Message#canonicalize(Any)
*
* @param msg message to produce
* @param pattern exchange pattern
*/
protected def produce(msg: Any, pattern: ExchangePattern): Unit = {
val cmsg = Message.canonicalize(msg)
val exchange = createExchange(pattern).fromRequestMessage(cmsg)
processor.process(exchange, new AsyncCallback {
val producer = self
// Need copies of sender and senderFuture references here
// since the callback could be done later by another thread.
val sender = self.sender
val senderFuture = self.senderFuture
def done(doneSync: Boolean): Unit = {
(doneSync, exchange.isFailed) match {
case (true, true) => dispatchSync(exchange.toFailureMessage(cmsg.headers(headersToCopy)))
case (true, false) => dispatchSync(exchange.toResponseMessage(cmsg.headers(headersToCopy)))
case (false, true) => dispatchAsync(FailureResult(exchange.toFailureMessage(cmsg.headers(headersToCopy))))
case (false, false) => dispatchAsync(MessageResult(exchange.toResponseMessage(cmsg.headers(headersToCopy))))
}
}
private def dispatchSync(result: Any) =
receiveAfterProduce(result)
private def dispatchAsync(result: Any) = {
if (senderFuture.isDefined)
producer.postMessageToMailboxAndCreateFutureResultWithTimeout(result, producer.timeout, sender, senderFuture)
else
producer.postMessageToMailbox(result, sender)
}
})
}
/**
* Produces <code>msg</code> to the endpoint specified by <code>endpointUri</code>. Before the message is
* actually sent it is pre-processed by calling <code>receiveBeforeProduce</code>. If <code>oneway</code>
* is <code>true</code>, an in-only message exchange is initiated, otherwise an in-out message exchange.
*
* @see Producer#produce(Any, ExchangePattern)
*/
protected def produce: Receive = {
case res: MessageResult => receiveAfterProduce(res.message)
case res: FailureResult => receiveAfterProduce(res.failure)
case msg => {
if (oneway)
produce(receiveBeforeProduce(msg), ExchangePattern.InOnly)
else
produce(receiveBeforeProduce(msg), ExchangePattern.InOut)
}
}
/**
* Called before the message is sent to the endpoint specified by <code>endpointUri</code>. The original
* message is passed as argument. By default, this method simply returns the argument but may be overridden
* by subtraits or subclasses.
*/
protected def receiveBeforeProduce: PartialFunction[Any, Any] = {
case msg => msg
}
/**
* Called after a response was received from the endpoint specified by <code>endpointUri</code>. The
* response is passed as argument. By default, this method sends the response back to the original sender
* if <code>oneway</code> is <code>false</code>. If <code>oneway</code> is <code>true</code>, nothing is
* done. This method may be overridden by subtraits or subclasses (e.g. to forward responses to another
* actor).
*/
protected def receiveAfterProduce: Receive = {
case msg => if (!oneway) self.reply(msg)
}
/**
* Creates a new Exchange of given <code>pattern</code> from the endpoint specified by
* <code>endpointUri</code>.
*/
private def createExchange(pattern: ExchangePattern): Exchange = endpoint.createExchange(pattern)
/**
* Creates a new <code>SendProcessor</code> for <code>endpoint</code>.
*/
private def createSendProcessor = {
val sendProcessor = new SendProcessor(endpoint)
sendProcessor.start
sendProcessor
}
}
/**
* Mixed in by Actor implementations to produce messages to Camel endpoints.
*/
trait Producer extends ProducerSupport { this: Actor =>
/**
* Default implementation of Actor.receive. Any messages received by this actors
* will be produced to the endpoint specified by <code>endpointUri</code>.
*/
protected def receive = produce
}
/**
* Java-friendly ProducerSupport.
*
* @see UntypedProducerActor
*
* @author Martin Krasser
*/
trait UntypedProducer extends ProducerSupport { this: UntypedActor =>
final override def endpointUri = getEndpointUri
final override def oneway = isOneway
final override def receiveBeforeProduce = {
case msg => onReceiveBeforeProduce(msg)
}
final override def receiveAfterProduce = {
case msg => onReceiveAfterProduce(msg)
}
/**
* Default implementation of UntypedActor.onReceive
*/
def onReceive(message: Any) = produce(message)
/**
* Returns the Camel endpoint URI to produce messages to.
*/
def getEndpointUri(): String
/**
* If set to false (default), this producer expects a response message from the Camel endpoint.
* If set to true, this producer communicates with the Camel endpoint with an in-only message
* exchange pattern (fire and forget).
*/
def isOneway() = super.oneway
/**
* Called before the message is sent to the endpoint specified by <code>getEndpointUri</code>. The original
* message is passed as argument. By default, this method simply returns the argument but may be overridden
* by subclasses.
*/
@throws(classOf[Exception])
def onReceiveBeforeProduce(message: Any): Any = super.receiveBeforeProduce(message)
/**
* Called after a response was received from the endpoint specified by <code>endpointUri</code>. The
* response is passed as argument. By default, this method sends the response back to the original sender
* if <code>oneway</code> is <code>false</code>. If <code>oneway</code> is <code>true</code>, nothing is
* done. This method may be overridden by subclasses (e.g. to forward responses to another actor).
*/
@throws(classOf[Exception])
def onReceiveAfterProduce(message: Any): Unit = super.receiveAfterProduce(message)
}
/**
* Subclass this abstract class to create an untyped producer actor. This class is meant to be used from Java.
*
* @author Martin Krasser
*/
abstract class UntypedProducerActor extends UntypedActor with UntypedProducer
/**
* @author Martin Krasser
*/
private[camel] case class MessageResult(message: Message)
/**
* @author Martin Krasser
*/
private[camel] case class FailureResult(failure: Failure)
/**
* A one-way producer.
*
* @author Martin Krasser
*/
trait Oneway extends Producer { this: Actor =>
override def oneway = true
}

View file

@ -1,305 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.camel.component
import java.net.InetSocketAddress
import java.util.{Map => JMap}
import java.util.concurrent.TimeoutException
import java.util.concurrent.atomic.AtomicReference
import org.apache.camel._
import org.apache.camel.impl.{DefaultProducer, DefaultEndpoint, DefaultComponent}
import akka.actor._
import akka.camel.{Failure, Message}
import akka.camel.CamelMessageConversion.toExchangeAdapter
import akka.dispatch.{CompletableFuture, MessageInvocation, MessageDispatcher}
import scala.reflect.BeanProperty
/**
* @author Martin Krasser
*/
object ActorComponent {
/**
* Name of the message header containing the actor id or uuid.
*/
val ActorIdentifier = "CamelActorIdentifier"
}
/**
* Camel component for sending messages to and receiving replies from (untyped) actors.
*
* @see akka.camel.component.ActorEndpoint
* @see akka.camel.component.ActorProducer
*
* @author Martin Krasser
*/
class ActorComponent extends DefaultComponent {
def createEndpoint(uri: String, remaining: String, parameters: JMap[String, Object]): ActorEndpoint = {
val (idType, idValue) = parsePath(remaining)
new ActorEndpoint(uri, this, idType, idValue)
}
private def parsePath(remaining: String): Tuple2[String, Option[String]] = remaining match {
case null | "" => throw new IllegalArgumentException("invalid path: [%s] - should be <actorid> or id:<actorid> or uuid:<actoruuid>" format remaining)
case id if id startsWith "id:" => ("id", parseIdentifier(id substring 3))
case uuid if uuid startsWith "uuid:" => ("uuid", parseIdentifier(uuid substring 5))
case id => ("id", parseIdentifier(id))
}
private def parseIdentifier(identifier: String): Option[String] =
if (identifier.length > 0) Some(identifier) else None
}
/**
* Camel endpoint for sending messages to and receiving replies from (untyped) actors. Actors
* are referenced using <code>actor</code> endpoint URIs of the following format:
* <code>actor:<actor-id></code>,
* <code>actor:id:[<actor-id>]</code> and
* <code>actor:uuid:[<actor-uuid>]</code>,
* where <code><actor-id></code> refers to <code>ActorRef.id</code> and <code><actor-uuid></code>
* refers to the String-representation od <code>ActorRef.uuid</code>. In URIs that contain
* <code>id:</code> or <code>uuid:</code>, an actor identifier (id or uuid) is optional. In this
* case, the in-message of an exchange produced to this endpoint must contain a message header
* with name <code>CamelActorIdentifier</code> and a value that is the target actor's identifier.
* If the URI contains an actor identifier, a message with a <code>CamelActorIdentifier</code>
* header overrides the identifier in the endpoint URI.
*
* @see akka.camel.component.ActorComponent
* @see akka.camel.component.ActorProducer
* @author Martin Krasser
*/
class ActorEndpoint(uri: String,
comp: ActorComponent,
val idType: String,
val idValue: Option[String]) extends DefaultEndpoint(uri, comp) {
/**
* Whether to block caller thread during two-way message exchanges with (untyped) actors. This is
* set via the <code>blocking=true|false</code> endpoint URI parameter. Default value is
* <code>false</code>.
*/
@BeanProperty var blocking: Boolean = false
/**
* @throws UnsupportedOperationException
*/
def createConsumer(processor: Processor): Consumer =
throw new UnsupportedOperationException("actor consumer not supported yet")
/**
* Creates a new ActorProducer instance initialized with this endpoint.
*/
def createProducer: ActorProducer = new ActorProducer(this)
/**
* Returns true.
*/
def isSingleton: Boolean = true
}
/**
* Sends the in-message of an exchange to an (untyped) actor, identified by an
* actor endpoint URI or by a <code>CamelActorIdentifier</code> message header.
* <ul>
* <li>If the exchange pattern is out-capable and <code>blocking</code> is set to
* <code>true</code> then the producer waits for a reply, using the !! operator.</li>
* <li>If the exchange pattern is out-capable and <code>blocking</code> is set to
* <code>false</code> then the producer sends the message using the ! operator, together
* with a callback handler. The callback handler is an <code>ActorRef</code> that can be
* used by the receiving actor to asynchronously reply to the route that is sending the
* message.</li>
* <li>If the exchange pattern is in-only then the producer sends the message using the
* ! operator.</li>
* </ul>
*
* @see akka.camel.component.ActorComponent
* @see akka.camel.component.ActorEndpoint
*
* @author Martin Krasser
*/
class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) with AsyncProcessor {
import ActorProducer._
private lazy val uuid = uuidFrom(ep.idValue.getOrElse(throw new ActorIdentifierNotSetException))
def process(exchange: Exchange) =
if (exchange.getPattern.isOutCapable) sendSync(exchange) else sendAsync(exchange)
def process(exchange: Exchange, callback: AsyncCallback): Boolean = {
(exchange.getPattern.isOutCapable, ep.blocking) match {
case (true, true) => {
sendSync(exchange)
callback.done(true)
true
}
case (true, false) => {
sendAsync(exchange, Some(AsyncCallbackAdapter(exchange, callback)))
false
}
case (false, _) => {
sendAsync(exchange)
callback.done(true)
true
}
}
}
private def sendSync(exchange: Exchange) = {
val actor = target(exchange)
val result: Any = actor !! requestFor(exchange)
result match {
case Some(msg: Failure) => exchange.fromFailureMessage(msg)
case Some(msg) => exchange.fromResponseMessage(Message.canonicalize(msg))
case None => throw new TimeoutException("timeout (%d ms) while waiting response from %s"
format (actor.timeout, ep.getEndpointUri))
}
}
private def sendAsync(exchange: Exchange, sender: Option[ActorRef] = None) =
target(exchange).!(requestFor(exchange))(sender)
private def target(exchange: Exchange) =
targetOption(exchange) getOrElse (throw new ActorNotRegisteredException(ep.getEndpointUri))
private def targetOption(exchange: Exchange): Option[ActorRef] = ep.idType match {
case "id" => targetById(targetId(exchange))
case "uuid" => targetByUuid(targetUuid(exchange))
}
private def targetId(exchange: Exchange) = exchange.getIn.getHeader(ActorComponent.ActorIdentifier) match {
case id: String => id
case null => ep.idValue.getOrElse(throw new ActorIdentifierNotSetException)
}
private def targetUuid(exchange: Exchange) = exchange.getIn.getHeader(ActorComponent.ActorIdentifier) match {
case uuid: Uuid => uuid
case uuid: String => uuidFrom(uuid)
case null => uuid
}
private def targetById(id: String) = ActorRegistry.actorsFor(id) match {
case actors if actors.length == 0 => None
case actors => Some(actors(0))
}
private def targetByUuid(uuid: Uuid) = ActorRegistry.actorFor(uuid)
}
/**
* @author Martin Krasser
*/
private[camel] object ActorProducer {
def requestFor(exchange: Exchange) =
exchange.toRequestMessage(Map(Message.MessageExchangeId -> exchange.getExchangeId))
}
/**
* Thrown to indicate that an actor referenced by an endpoint URI cannot be
* found in the ActorRegistry.
*
* @author Martin Krasser
*/
class ActorNotRegisteredException(uri: String) extends RuntimeException {
override def getMessage = "%s not registered" format uri
}
/**
* Thrown to indicate that no actor identifier has been set.
*
* @author Martin Krasser
*/
class ActorIdentifierNotSetException extends RuntimeException {
override def getMessage = "actor identifier not set"
}
/**
* @author Martin Krasser
*/
private[akka] object AsyncCallbackAdapter {
/**
* Creates and starts an <code>AsyncCallbackAdapter</code>.
*
* @param exchange message exchange to write results to.
* @param callback callback object to generate completion notifications.
*/
def apply(exchange: Exchange, callback: AsyncCallback) =
new AsyncCallbackAdapter(exchange, callback).start
}
/**
* Adapts an <code>ActorRef</code> to a Camel <code>AsyncCallback</code>. Used by receiving actors to reply
* asynchronously to Camel routes with <code>ActorRef.reply</code>.
* <p>
* <em>Please note</em> that this adapter can only be used locally at the moment which should not
* be a problem is most situations since Camel endpoints are only activated for local actor references,
* never for remote references.
*
* @author Martin Krasser
*/
private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCallback) extends ActorRef with ScalaActorRef {
def start = {
_status = ActorRefInternals.RUNNING
this
}
def stop() = {
_status = ActorRefInternals.SHUTDOWN
}
/**
* Populates the initial <code>exchange</code> with the reply <code>message</code> and uses the
* <code>callback</code> handler to notify Camel about the asynchronous completion of the message
* exchange.
*
* @param message reply message
* @param sender ignored
*/
protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]) = {
message match {
case msg: Failure => exchange.fromFailureMessage(msg)
case msg => exchange.fromResponseMessage(Message.canonicalize(msg))
}
callback.done(false)
}
def actorClass: Class[_ <: Actor] = unsupported
def actorClassName = unsupported
def dispatcher_=(md: MessageDispatcher): Unit = unsupported
def dispatcher: MessageDispatcher = unsupported
def makeRemote(hostname: String, port: Int): Unit = unsupported
def makeRemote(address: InetSocketAddress): Unit = unsupported
def homeAddress_=(address: InetSocketAddress): Unit = unsupported
def remoteAddress: Option[InetSocketAddress] = unsupported
def link(actorRef: ActorRef): Unit = unsupported
def unlink(actorRef: ActorRef): Unit = unsupported
def startLink(actorRef: ActorRef): Unit = unsupported
def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = unsupported
def spawn(clazz: Class[_ <: Actor]): ActorRef = unsupported
def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
def spawnLink(clazz: Class[_ <: Actor]): ActorRef = unsupported
def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
def shutdownLinkedActors: Unit = unsupported
def supervisor: Option[ActorRef] = unsupported
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T](message: Any, timeout: Long, senderOption: Option[ActorRef], senderFuture: Option[CompletableFuture[T]]) = unsupported
protected[akka] def mailbox: AnyRef = unsupported
protected[akka] def mailbox_=(msg: AnyRef):AnyRef = unsupported
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
protected[akka] def linkedActors: JMap[Uuid, ActorRef] = unsupported
protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported
protected[akka] def registerSupervisorAsRemoteActor = unsupported
protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = unsupported
protected[akka] def actorInstance: AtomicReference[Actor] = unsupported
private def unsupported = throw new UnsupportedOperationException("Not supported for %s" format classOf[AsyncCallbackAdapter].getName)
}

View file

@ -1,111 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.camel.component
import java.util.Map
import java.util.concurrent.ConcurrentHashMap
import org.apache.camel.CamelContext
import org.apache.camel.component.bean._
/**
* @author Martin Krasser
*/
object TypedActorComponent {
/**
* Default schema name for typed actor endpoint URIs.
*/
val InternalSchema = "typed-actor-internal"
}
/**
* Camel component for exchanging messages with typed actors. This component
* tries to obtain the typed actor from its <code>typedActorRegistry</code>
* first. If it's not there it tries to obtain it from the CamelContext's registry.
*
* @see org.apache.camel.component.bean.BeanComponent
*
* @author Martin Krasser
*/
class TypedActorComponent extends BeanComponent {
val typedActorRegistry = new ConcurrentHashMap[String, AnyRef]
/**
* Creates an <code>org.apache.camel.component.bean.BeanEndpoint</code> with a custom
* bean holder that uses <code>typedActorRegistry</code> for getting access to typed
* actors (beans).
*
* @see akka.camel.component.TypedActorHolder
*/
override def createEndpoint(uri: String, remaining: String, parameters: Map[String, AnyRef]) = {
val endpoint = new BeanEndpoint(uri, this)
endpoint.setBeanName(remaining)
endpoint.setBeanHolder(createBeanHolder(remaining))
setProperties(endpoint.getProcessor, parameters)
endpoint
}
private def createBeanHolder(beanName: String) =
new TypedActorHolder(typedActorRegistry, getCamelContext, beanName).createCacheHolder
}
/**
* <code>org.apache.camel.component.bean.BeanHolder</code> implementation that uses a custom
* registry for getting access to typed actors.
*
* @author Martin Krasser
*/
class TypedActorHolder(typedActorRegistry: Map[String, AnyRef], context: CamelContext, name: String)
extends RegistryBean(context, name) {
/**
* Returns an <code>akka.camel.component.TypedActorInfo</code> instance.
*/
override def getBeanInfo: BeanInfo =
new TypedActorInfo(getContext, getBean.getClass, getParameterMappingStrategy)
/**
* Obtains a typed actor from <code>typedActorRegistry</code>. If the typed actor cannot
* be found then this method tries to obtain the actor from the CamelContext's registry.
*
* @return a typed actor or <code>null</code>.
*/
override def getBean: AnyRef = {
val bean = typedActorRegistry.get(getName)
if (bean eq null) super.getBean else bean
}
}
/**
* Typed actor meta information.
*
* @author Martin Krasser
*/
class TypedActorInfo(context: CamelContext, clazz: Class[_], strategy: ParameterMappingStrategy)
extends BeanInfo(context, clazz, strategy) {
/**
* Introspects AspectWerkz proxy classes.
*
* @param clazz AspectWerkz proxy class.
*/
protected override def introspect(clazz: Class[_]): Unit = {
// TODO: fix target class detection in BeanInfo.introspect(Class)
// Camel assumes that classes containing a '$$' in the class name
// are classes generated with CGLIB. This conflicts with proxies
// created from interfaces with AspectWerkz. Once the fix is in
// place this method can be removed.
for (method <- clazz.getDeclaredMethods) {
if (isValidMethod(clazz, method)) {
introspect(clazz, method)
}
}
val superclass = clazz.getSuperclass
if ((superclass ne null) && !superclass.equals(classOf[AnyRef])) {
introspect(superclass)
}
}
}

View file

@ -1,59 +0,0 @@
package akka.camel;
import akka.actor.ActorRegistry;
import akka.actor.TypedActor;
import akka.actor.UntypedActor;
import akka.japi.SideEffect;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import static akka.camel.CamelContextManager.*;
import static akka.camel.CamelServiceManager.*;
import static org.junit.Assert.*;
/**
* @author Martin Krasser
*/
public class ConsumerJavaTestBase {
private SampleErrorHandlingTypedConsumer consumer;
@BeforeClass
public static void setUpBeforeClass() {
startCamelService();
}
@AfterClass
public static void tearDownAfterClass() {
stopCamelService();
ActorRegistry.shutdownAll();
}
@Test
public void shouldHandleExceptionThrownByActorAndGenerateCustomResponse() {
getMandatoryService().awaitEndpointActivation(1, new SideEffect() {
public void apply() {
UntypedActor.actorOf(SampleErrorHandlingConsumer.class).start();
}
});
String result = getMandatoryTemplate().requestBody("direct:error-handler-test-java", "hello", String.class);
assertEquals("error: hello", result);
}
@Test
public void shouldHandleExceptionThrownByTypedActorAndGenerateCustomResponse() {
getMandatoryService().awaitEndpointActivation(1, new SideEffect() {
public void apply() {
consumer = TypedActor.newInstance(
SampleErrorHandlingTypedConsumer.class,
SampleErrorHandlingTypedConsumerImpl.class);
}
});
String result = getMandatoryTemplate().requestBody("direct:error-handler-test-java-typed", "hello", String.class);
assertEquals("error: hello", result);
}
}

View file

@ -1,129 +0,0 @@
package akka.camel;
import org.apache.camel.NoTypeConversionAvailableException;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.camel.CamelContextManager;
import akka.camel.Message;
import akka.japi.Function;
import java.io.InputStream;
import java.util.*;
import static org.junit.Assert.*;
/**
* @author Martin Krasser
*/
public class MessageJavaTestBase {
@BeforeClass
public static void setUpBeforeClass() {
CamelContextManager.init();
}
@Test public void shouldConvertDoubleBodyToString() {
assertEquals("1.4", new Message("1.4").getBodyAs(String.class));
}
@Test(expected=NoTypeConversionAvailableException.class)
public void shouldThrowExceptionWhenConvertingDoubleBodyToInputStream() {
new Message(1.4).getBodyAs(InputStream.class);
}
@Test public void shouldReturnDoubleHeader() {
Message message = new Message("test" , createMap("test", 1.4));
assertEquals(1.4, message.getHeader("test"));
}
@Test public void shouldConvertDoubleHeaderToString() {
Message message = new Message("test" , createMap("test", 1.4));
assertEquals("1.4", message.getHeaderAs("test", String.class));
}
@Test public void shouldReturnSubsetOfHeaders() {
Message message = new Message("test" , createMap("A", "1", "B", "2"));
assertEquals(createMap("B", "2"), message.getHeaders(createSet("B")));
}
@Test(expected=UnsupportedOperationException.class)
public void shouldReturnSubsetOfHeadersUnmodifiable() {
Message message = new Message("test" , createMap("A", "1", "B", "2"));
message.getHeaders(createSet("B")).put("x", "y");
}
@Test public void shouldReturnAllHeaders() {
Message message = new Message("test" , createMap("A", "1", "B", "2"));
assertEquals(createMap("A", "1", "B", "2"), message.getHeaders());
}
@Test(expected=UnsupportedOperationException.class)
public void shouldReturnAllHeadersUnmodifiable() {
Message message = new Message("test" , createMap("A", "1", "B", "2"));
message.getHeaders().put("x", "y");
}
@Test public void shouldTransformBodyAndPreserveHeaders() {
assertEquals(
new Message("ab", createMap("A", "1")),
new Message("a" , createMap("A", "1")).transformBody((Function)new TestTransformer()));
}
@Test public void shouldConvertBodyAndPreserveHeaders() {
assertEquals(
new Message("1.4", createMap("A", "1")),
new Message(1.4 , createMap("A", "1")).setBodyAs(String.class));
}
@Test public void shouldSetBodyAndPreserveHeaders() {
assertEquals(
new Message("test2" , createMap("A", "1")),
new Message("test1" , createMap("A", "1")).setBody("test2"));
}
@Test public void shouldSetHeadersAndPreserveBody() {
assertEquals(
new Message("test1" , createMap("C", "3")),
new Message("test1" , createMap("A", "1")).setHeaders(createMap("C", "3")));
}
@Test public void shouldAddHeaderAndPreserveBodyAndHeaders() {
assertEquals(
new Message("test1" , createMap("A", "1", "B", "2")),
new Message("test1" , createMap("A", "1")).addHeader("B", "2"));
}
@Test public void shouldAddHeadersAndPreserveBodyAndHeaders() {
assertEquals(
new Message("test1" , createMap("A", "1", "B", "2")),
new Message("test1" , createMap("A", "1")).addHeaders(createMap("B", "2")));
}
@Test public void shouldRemoveHeadersAndPreserveBodyAndRemainingHeaders() {
assertEquals(
new Message("test1" , createMap("A", "1")),
new Message("test1" , createMap("A", "1", "B", "2")).removeHeader("B"));
}
private static Set<String> createSet(String... entries) {
HashSet<String> set = new HashSet<String>();
set.addAll(Arrays.asList(entries));
return set;
}
private static Map<String, Object> createMap(Object... pairs) {
HashMap<String, Object> map = new HashMap<String, Object>();
for (int i = 0; i < pairs.length; i += 2) {
map.put((String)pairs[i], pairs[i+1]);
}
return map;
}
private static class TestTransformer implements Function<String, String> {
public String apply(String param) {
return param + "b";
}
}
}

View file

@ -1,34 +0,0 @@
package akka.camel;
import org.apache.camel.builder.Builder;
import org.apache.camel.model.ProcessorDefinition;
import org.apache.camel.model.RouteDefinition;
/**
* @author Martin Krasser
*/
public class SampleErrorHandlingConsumer extends UntypedConsumerActor {
public String getEndpointUri() {
return "direct:error-handler-test-java";
}
public boolean isBlocking() {
return true;
}
public void preStart() {
onRouteDefinition(new RouteDefinitionHandler() {
public ProcessorDefinition<?> onRouteDefinition(RouteDefinition rd) {
return rd.onException(Exception.class).handled(true).transform(Builder.exceptionMessage()).end();
}
});
}
public void onReceive(Object message) throws Exception {
Message msg = (Message)message;
String body = msg.getBodyAs(String.class);
throw new Exception(String.format("error: %s", body));
}
}

View file

@ -1,11 +0,0 @@
package akka.camel;
/**
* @author Martin Krasser
*/
public interface SampleErrorHandlingTypedConsumer {
@consume(value="direct:error-handler-test-java-typed", routeDefinitionHandler=SampleRouteDefinitionHandler.class)
String willFail(String s);
}

View file

@ -1,14 +0,0 @@
package akka.camel;
import akka.actor.TypedActor;
/**
* @author Martin Krasser
*/
public class SampleErrorHandlingTypedConsumerImpl extends TypedActor implements SampleErrorHandlingTypedConsumer {
public String willFail(String s) {
throw new RuntimeException(String.format("error: %s", s));
}
}

View file

@ -1,12 +0,0 @@
package akka.camel;
import akka.camel.consume;
/**
* @author Martin Krasser
*/
public interface SampleRemoteTypedConsumer {
@consume("direct:remote-typed-consumer")
public String foo(String s);
}

View file

@ -1,14 +0,0 @@
package akka.camel;
import akka.actor.TypedActor;
/**
* @author Martin Krasser
*/
public class SampleRemoteTypedConsumerImpl extends TypedActor implements SampleRemoteTypedConsumer {
public String foo(String s) {
return String.format("remote typed actor: %s", s);
}
}

View file

@ -1,29 +0,0 @@
package akka.camel;
import akka.camel.RemoteUntypedConsumerActor;
/**
* @author Martin Krasser
*/
public class SampleRemoteUntypedConsumer extends RemoteUntypedConsumerActor {
public SampleRemoteUntypedConsumer() {
this("localhost", 7774);
}
public SampleRemoteUntypedConsumer(String host, int port) {
super(host, port);
}
public String getEndpointUri() {
return "direct:remote-untyped-consumer";
}
public void onReceive(Object message) {
Message msg = (Message)message;
String body = msg.getBodyAs(String.class);
String header = msg.getHeaderAs("test", String.class);
getContext().replySafe(String.format("%s %s", body, header));
}
}

View file

@ -1,14 +0,0 @@
package akka.camel;
import org.apache.camel.builder.Builder;
import org.apache.camel.model.ProcessorDefinition;
import org.apache.camel.model.RouteDefinition;
/**
* @author Martin Krasser
*/
public class SampleRouteDefinitionHandler implements RouteDefinitionHandler {
public ProcessorDefinition<?> onRouteDefinition(RouteDefinition rd) {
return rd.onException(Exception.class).handled(true).transform(Builder.exceptionMessage()).end();
}
}

View file

@ -1,9 +0,0 @@
package akka.camel;
/**
* @author Martin Krasser
*/
public interface SampleTypedActor {
public String foo(String s);
}

View file

@ -1,14 +0,0 @@
package akka.camel;
import akka.actor.TypedActor;
/**
* @author Martin Krasser
*/
public class SampleTypedActorImpl extends TypedActor implements SampleTypedActor {
public String foo(String s) {
return String.format("foo: %s", s);
}
}

View file

@ -1,20 +0,0 @@
package akka.camel;
import org.apache.camel.Body;
import org.apache.camel.Header;
import akka.camel.consume;
/**
* @author Martin Krasser
*/
public interface SampleTypedConsumer {
public String m1(String b, String h);
public String m2(@Body String b, @Header("test") String h);
public String m3(@Body String b, @Header("test") String h);
@consume("direct:m4")
public String m4(@Body String b, @Header("test") String h);
public void m5(@Body String b, @Header("test") String h);
}

View file

@ -1,30 +0,0 @@
package akka.camel;
import akka.actor.TypedActor;
/**
* @author Martin Krasser
*/
public class SampleTypedConsumerImpl extends TypedActor implements SampleTypedConsumer {
public String m1(String b, String h) {
return "m1: " + b + " " + h;
}
@consume("direct:m2")
public String m2(String b, String h) {
return "m2: " + b + " " + h;
}
@consume("direct:m3")
public String m3(String b, String h) {
return "m3: " + b + " " + h;
}
public String m4(String b, String h) {
return "m4: " + b + " " + h;
}
public void m5(String b, String h) {
}
}

View file

@ -1,13 +0,0 @@
package akka.camel;
import akka.camel.consume;
/**
* @author Martin Krasser
*/
public interface SampleTypedSingleConsumer {
@consume("direct:foo")
public void foo(String b);
}

View file

@ -1,13 +0,0 @@
package akka.camel;
import akka.actor.TypedActor;
/**
* @author Martin Krasser
*/
public class SampleTypedSingleConsumerImpl extends TypedActor implements SampleTypedSingleConsumer {
public void foo(String b) {
}
}

View file

@ -1,12 +0,0 @@
package akka.camel;
import akka.actor.UntypedActor;
/**
* @author Martin Krasser
*/
public class SampleUntypedActor extends UntypedActor {
public void onReceive(Object message) {
logger().debug("Yay! I haz a message!");
}
}

View file

@ -1,21 +0,0 @@
package akka.camel;
import akka.camel.UntypedConsumerActor;
/**
* @author Martin Krasser
*/
public class SampleUntypedConsumer extends UntypedConsumerActor {
public String getEndpointUri() {
return "direct:test-untyped-consumer";
}
public void onReceive(Object message) {
Message msg = (Message)message;
String body = msg.getBodyAs(String.class);
String header = msg.getHeaderAs("test", String.class);
getContext().replySafe(String.format("%s %s", body, header));
}
}

View file

@ -1,23 +0,0 @@
package akka.camel;
/**
* @author Martin Krasser
*/
public class SampleUntypedConsumerBlocking extends UntypedConsumerActor {
public String getEndpointUri() {
return "direct:test-untyped-consumer-blocking";
}
public boolean isBlocking() {
return true;
}
public void onReceive(Object message) {
Message msg = (Message)message;
String body = msg.getBodyAs(String.class);
String header = msg.getHeaderAs("test", String.class);
getContext().replySafe(String.format("%s %s", body, header));
}
}

View file

@ -1,18 +0,0 @@
package akka.camel;
/**
* @author Martin Krasser
*/
public class SampleUntypedForwardingProducer extends UntypedProducerActor {
public String getEndpointUri() {
return "direct:producer-test-1";
}
@Override
public void onReceiveAfterProduce(Object message) {
Message msg = (Message)message;
String body = msg.getBodyAs(String.class);
CamelContextManager.getMandatoryTemplate().sendBody("direct:forward-test-1", body);
}
}

View file

@ -1,12 +0,0 @@
package akka.camel;
/**
* @author Martin Krasser
*/
public class SampleUntypedReplyingProducer extends UntypedProducerActor {
public String getEndpointUri() {
return "direct:producer-test-1";
}
}

View file

@ -1,36 +0,0 @@
package akka.camel
import org.apache.camel.impl.{DefaultProducerTemplate, DefaultCamelContext}
import org.junit.Test
import org.scalatest.junit.JUnitSuite
class CamelContextLifecycleTest extends JUnitSuite with CamelContextLifecycle {
@Test def shouldManageCustomCamelContext {
assert(context === None)
assert(template === None)
intercept[IllegalStateException] { mandatoryContext }
intercept[IllegalStateException] { mandatoryTemplate }
val ctx = new TestCamelContext
assert(ctx.isStreamCaching === false)
init(ctx)
assert(mandatoryContext.isStreamCaching === true)
assert(!mandatoryContext.asInstanceOf[TestCamelContext].isStarted)
assert(mandatoryTemplate.asInstanceOf[DefaultProducerTemplate].isStarted)
start
assert(mandatoryContext.asInstanceOf[TestCamelContext].isStarted)
assert(mandatoryTemplate.asInstanceOf[DefaultProducerTemplate].isStarted)
stop
assert(!mandatoryContext.asInstanceOf[TestCamelContext].isStarted)
assert(!mandatoryTemplate.asInstanceOf[DefaultProducerTemplate].isStarted)
}
class TestCamelContext extends DefaultCamelContext
}

View file

@ -1,109 +0,0 @@
package akka.camel
import org.apache.camel.impl.{DefaultCamelContext, DefaultExchange}
import org.apache.camel.ExchangePattern
import org.junit.Test
import org.scalatest.junit.JUnitSuite
class CamelExchangeAdapterTest extends JUnitSuite {
import CamelMessageConversion.toExchangeAdapter
@Test def shouldSetInMessageFromRequestMessage = {
val e1 = sampleInOnly.fromRequestMessage(Message("x"))
assert(e1.getIn.getBody === "x")
val e2 = sampleInOut.fromRequestMessage(Message("y"))
assert(e2.getIn.getBody === "y")
}
@Test def shouldSetOutMessageFromResponseMessage = {
val e1 = sampleInOut.fromResponseMessage(Message("y"))
assert(e1.getOut.getBody === "y")
}
@Test def shouldSetInMessageFromResponseMessage = {
val e1 = sampleInOnly.fromResponseMessage(Message("x"))
assert(e1.getIn.getBody === "x")
}
@Test def shouldSetExceptionFromFailureMessage = {
val e1 = sampleInOnly.fromFailureMessage(Failure(new Exception("test1")))
assert(e1.getException.getMessage === "test1")
val e2 = sampleInOut.fromFailureMessage(Failure(new Exception("test2")))
assert(e2.getException.getMessage === "test2")
}
@Test def shouldCreateRequestMessageFromInMessage = {
val m = sampleInOnly.toRequestMessage
assert(m === Message("test-in", Map("key-in" -> "val-in")))
}
@Test def shouldCreateResponseMessageFromInMessage = {
val m = sampleInOnly.toResponseMessage
assert(m === Message("test-in", Map("key-in" -> "val-in")))
}
@Test def shouldCreateResponseMessageFromOutMessage = {
val m = sampleInOut.toResponseMessage
assert(m === Message("test-out", Map("key-out" -> "val-out")))
}
@Test def shouldCreateFailureMessageFromExceptionAndInMessage = {
val e1 = sampleInOnly
e1.setException(new Exception("test1"))
assert(e1.toFailureMessage.cause.getMessage === "test1")
assert(e1.toFailureMessage.headers("key-in") === "val-in")
}
@Test def shouldCreateFailureMessageFromExceptionAndOutMessage = {
val e1 = sampleInOut
e1.setException(new Exception("test2"))
assert(e1.toFailureMessage.cause.getMessage === "test2")
assert(e1.toFailureMessage.headers("key-out") === "val-out")
}
@Test def shouldCreateRequestMessageFromInMessageWithAdditionalHeader = {
val m = sampleInOnly.toRequestMessage(Map("x" -> "y"))
assert(m === Message("test-in", Map("key-in" -> "val-in", "x" -> "y")))
}
@Test def shouldCreateResponseMessageFromInMessageWithAdditionalHeader = {
val m = sampleInOnly.toResponseMessage(Map("x" -> "y"))
assert(m === Message("test-in", Map("key-in" -> "val-in", "x" -> "y")))
}
@Test def shouldCreateResponseMessageFromOutMessageWithAdditionalHeader = {
val m = sampleInOut.toResponseMessage(Map("x" -> "y"))
assert(m === Message("test-out", Map("key-out" -> "val-out", "x" -> "y")))
}
@Test def shouldCreateFailureMessageFromExceptionAndInMessageWithAdditionalHeader = {
val e1 = sampleInOnly
e1.setException(new Exception("test1"))
assert(e1.toFailureMessage.cause.getMessage === "test1")
val headers = e1.toFailureMessage(Map("x" -> "y")).headers
assert(headers("key-in") === "val-in")
assert(headers("x") === "y")
}
@Test def shouldCreateFailureMessageFromExceptionAndOutMessageWithAdditionalHeader = {
val e1 = sampleInOut
e1.setException(new Exception("test2"))
assert(e1.toFailureMessage.cause.getMessage === "test2")
val headers = e1.toFailureMessage(Map("x" -> "y")).headers
assert(headers("key-out") === "val-out")
assert(headers("x") === "y")
}
private def sampleInOnly = sampleExchange(ExchangePattern.InOnly)
private def sampleInOut = sampleExchange(ExchangePattern.InOut)
private def sampleExchange(pattern: ExchangePattern) = {
val exchange = new DefaultExchange(new DefaultCamelContext)
exchange.getIn.setBody("test-in")
exchange.getOut.setBody("test-out")
exchange.getIn.setHeader("key-in", "val-in")
exchange.getOut.setHeader("key-out", "val-out")
exchange.setPattern(pattern)
exchange
}
}

View file

@ -1,38 +0,0 @@
package akka.camel
import org.apache.camel.impl.DefaultMessage
import org.junit.Test
import org.scalatest.junit.JUnitSuite
class CamelMessageAdapterTest extends JUnitSuite {
import CamelMessageConversion.toMessageAdapter
@Test def shouldOverwriteBodyAndAddHeader = {
val cm = sampleMessage.fromMessage(Message("blah", Map("key" -> "baz")))
assert(cm.getBody === "blah")
assert(cm.getHeader("foo") === "bar")
assert(cm.getHeader("key") === "baz")
}
@Test def shouldCreateMessageWithBodyAndHeader = {
val m = sampleMessage.toMessage
assert(m.body === "test")
assert(m.headers("foo") === "bar")
}
@Test def shouldCreateMessageWithBodyAndHeaderAndCustomHeader = {
val m = sampleMessage.toMessage(Map("key" -> "baz"))
assert(m.body === "test")
assert(m.headers("foo") === "bar")
assert(m.headers("key") === "baz")
}
private[camel] def sampleMessage = {
val message = new DefaultMessage
message.setBody("test")
message.setHeader("foo", "bar")
message
}
}

View file

@ -1,62 +0,0 @@
package akka.camel
import org.scalatest.{BeforeAndAfterAll, WordSpec}
import org.scalatest.matchers.MustMatchers
import akka.actor.ActorRegistry
/**
* @author Martin Krasser
*/
class CamelServiceManagerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
override def afterAll = {
CamelServiceManager.stopCamelService
ActorRegistry.shutdownAll
}
"A CamelServiceManager" when {
"the startCamelService method been has been called" must {
"have registered the started CamelService instance" in {
val service = CamelServiceManager.startCamelService
CamelServiceManager.mandatoryService must be theSameInstanceAs (service)
}
}
"the stopCamelService method been has been called" must {
"have unregistered the current CamelService instance" in {
val service = CamelServiceManager.stopCamelService
CamelServiceManager.service must be (None)
}
}
}
"A CamelServiceManager" when {
val service = CamelServiceFactory.createCamelService
"a CamelService instance has been started externally" must {
"have registered the started CamelService instance" in {
service.start
CamelServiceManager.mandatoryService must be theSameInstanceAs (service)
}
}
"the current CamelService instance has been stopped externally" must {
"have unregistered the current CamelService instance" in {
service.stop
CamelServiceManager.service must be (None)
}
}
}
"A CamelServiceManager" when {
"a CamelService has been started" must {
"not allow further CamelService instances to be started" in {
CamelServiceManager.startCamelService
intercept[IllegalStateException] { CamelServiceManager.startCamelService }
}
}
"a CamelService has been stopped" must {
"only allow the current CamelService instance to be stopped" in {
intercept[IllegalStateException] { CamelServiceFactory.createCamelService.stop }
}
}
}
}

View file

@ -1,5 +0,0 @@
package akka.camel
import org.scalatest.junit.JUnitSuite
class ConsumerJavaTest extends ConsumerJavaTestBase with JUnitSuite

View file

@ -1,63 +0,0 @@
package akka.camel
import org.junit.Test
import org.scalatest.junit.JUnitSuite
import akka.actor.{ActorRef, Actor, UntypedActor}
class ConsumerRegisteredTest extends JUnitSuite {
import ConsumerRegisteredTest._
@Test def shouldCreateSomeNonBlockingPublishRequestFromConsumer = {
val c = Actor.actorOf[ConsumerActor1]
val event = ConsumerActorRegistered.forConsumer(c)
assert(event === Some(ConsumerActorRegistered(c, consumerOf(c))))
}
@Test def shouldCreateSomeBlockingPublishRequestFromConsumer = {
val c = Actor.actorOf[ConsumerActor2]
val event = ConsumerActorRegistered.forConsumer(c)
assert(event === Some(ConsumerActorRegistered(c, consumerOf(c))))
}
@Test def shouldCreateNoneFromConsumer = {
val event = ConsumerActorRegistered.forConsumer(Actor.actorOf[PlainActor])
assert(event === None)
}
@Test def shouldCreateSomeNonBlockingPublishRequestFromUntypedConsumer = {
val uc = UntypedActor.actorOf(classOf[SampleUntypedConsumer])
val event = ConsumerActorRegistered.forConsumer(uc)
assert(event === Some(ConsumerActorRegistered(uc, consumerOf(uc))))
}
@Test def shouldCreateSomeBlockingPublishRequestFromUntypedConsumer = {
val uc = UntypedActor.actorOf(classOf[SampleUntypedConsumerBlocking])
val event = ConsumerActorRegistered.forConsumer(uc)
assert(event === Some(ConsumerActorRegistered(uc, consumerOf(uc))))
}
@Test def shouldCreateNoneFromUntypedConsumer = {
val a = UntypedActor.actorOf(classOf[SampleUntypedActor])
val event = ConsumerActorRegistered.forConsumer(a)
assert(event === None)
}
private def consumerOf(ref: ActorRef) = ref.actor.asInstanceOf[Consumer]
}
object ConsumerRegisteredTest {
class ConsumerActor1 extends Actor with Consumer {
def endpointUri = "mock:test1"
protected def receive = null
}
class ConsumerActor2 extends Actor with Consumer {
def endpointUri = "mock:test2"
override def blocking = true
protected def receive = null
}
class PlainActor extends Actor {
protected def receive = null
}
}

View file

@ -1,271 +0,0 @@
package akka.camel
import java.util.concurrent.{TimeoutException, CountDownLatch, TimeUnit}
import org.apache.camel.CamelExecutionException
import org.apache.camel.builder.Builder
import org.apache.camel.model.RouteDefinition
import org.scalatest.{BeforeAndAfterAll, WordSpec}
import org.scalatest.matchers.MustMatchers
import akka.actor.Actor._
import akka.actor._
/**
* @author Martin Krasser
*/
class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
import CamelContextManager.mandatoryTemplate
import ConsumerScalaTest._
var service: CamelService = _
override protected def beforeAll = {
ActorRegistry.shutdownAll
// create new CamelService instance
service = CamelServiceFactory.createCamelService
// Register publish requestor as listener
service.registerPublishRequestor
// register test consumer before starting the CamelService
actorOf(new TestConsumer("direct:publish-test-1")).start
// start consumer publisher, otherwise we cannot set message
// count expectations in the next step (needed for testing only).
service.consumerPublisher.start
service.awaitEndpointActivation(1) {
service.start
} must be (true)
}
override protected def afterAll = {
service.stop
ActorRegistry.shutdownAll
}
"A responding consumer" when {
val consumer = actorOf(new TestConsumer("direct:publish-test-2"))
"started before starting the CamelService" must {
"support an in-out message exchange via its endpoint" in {
mandatoryTemplate.requestBody("direct:publish-test-1", "msg1") must equal ("received msg1")
}
}
"not started" must {
"not have an associated endpoint in the CamelContext" in {
CamelContextManager.mandatoryContext.hasEndpoint("direct:publish-test-2") must be (null)
}
}
"started" must {
"support an in-out message exchange via its endpoint" in {
service.awaitEndpointActivation(1) {
consumer.start
} must be (true)
mandatoryTemplate.requestBody("direct:publish-test-2", "msg2") must equal ("received msg2")
}
"have an associated endpoint in the CamelContext" in {
CamelContextManager.mandatoryContext.hasEndpoint("direct:publish-test-2") must not be (null)
}
}
"stopped" must {
"not support an in-out message exchange via its endpoint" in {
service.awaitEndpointDeactivation(1) {
consumer.stop
} must be (true)
intercept[CamelExecutionException] {
mandatoryTemplate.requestBody("direct:publish-test-2", "msg2")
}
}
}
}
"A responding, typed consumer" when {
var actor: SampleTypedConsumer = null
"started" must {
"support in-out message exchanges via its endpoints" in {
service.awaitEndpointActivation(3) {
actor = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
} must be (true)
mandatoryTemplate.requestBodyAndHeader("direct:m2", "x", "test", "y") must equal ("m2: x y")
mandatoryTemplate.requestBodyAndHeader("direct:m3", "x", "test", "y") must equal ("m3: x y")
mandatoryTemplate.requestBodyAndHeader("direct:m4", "x", "test", "y") must equal ("m4: x y")
}
}
"stopped" must {
"not support in-out message exchanges via its endpoints" in {
service.awaitEndpointDeactivation(3) {
TypedActor.stop(actor)
} must be (true)
intercept[CamelExecutionException] {
mandatoryTemplate.requestBodyAndHeader("direct:m2", "x", "test", "y")
}
intercept[CamelExecutionException] {
mandatoryTemplate.requestBodyAndHeader("direct:m3", "x", "test", "y")
}
intercept[CamelExecutionException] {
mandatoryTemplate.requestBodyAndHeader("direct:m4", "x", "test", "y")
}
}
}
}
"A responding, typed consumer (Scala)" when {
var actor: TestTypedConsumer = null
"started" must {
"support in-out message exchanges via its endpoints" in {
service.awaitEndpointActivation(2) {
actor = TypedActor.newInstance(classOf[TestTypedConsumer], classOf[TestTypedConsumerImpl])
} must be (true)
mandatoryTemplate.requestBody("direct:publish-test-3", "x") must equal ("foo: x")
mandatoryTemplate.requestBody("direct:publish-test-4", "x") must equal ("bar: x")
}
}
"stopped" must {
"not support in-out message exchanges via its endpoints" in {
service.awaitEndpointDeactivation(2) {
TypedActor.stop(actor)
} must be (true)
intercept[CamelExecutionException] {
mandatoryTemplate.requestBody("direct:publish-test-3", "x")
}
intercept[CamelExecutionException] {
mandatoryTemplate.requestBody("direct:publish-test-4", "x")
}
}
}
}
"A responding, untyped consumer" when {
val consumer = UntypedActor.actorOf(classOf[SampleUntypedConsumer])
"started" must {
"support an in-out message exchange via its endpoint" in {
service.awaitEndpointActivation(1) {
consumer.start
} must be (true)
mandatoryTemplate.requestBodyAndHeader("direct:test-untyped-consumer", "x", "test", "y") must equal ("x y")
}
}
"stopped" must {
"not support an in-out message exchange via its endpoint" in {
service.awaitEndpointDeactivation(1) {
consumer.stop
} must be (true)
intercept[CamelExecutionException] {
mandatoryTemplate.sendBodyAndHeader("direct:test-untyped-consumer", "blah", "test", "blub")
}
}
}
}
"A non-responding, blocking consumer" when {
"receiving an in-out message exchange" must {
"lead to a TimeoutException" in {
service.awaitEndpointActivation(1) {
actorOf(new TestBlocker("direct:publish-test-5")).start
} must be (true)
try {
mandatoryTemplate.requestBody("direct:publish-test-5", "msg3")
fail("expected TimoutException not thrown")
} catch {
case e => {
assert(e.getCause.isInstanceOf[TimeoutException])
}
}
}
}
}
"A responding, blocking consumer" when {
"activated with a custom error handler" must {
"handle thrown exceptions by generating a custom response" in {
service.awaitEndpointActivation(1) {
actorOf[ErrorHandlingConsumer].start
} must be (true)
mandatoryTemplate.requestBody("direct:error-handler-test", "hello") must equal ("error: hello")
}
}
"activated with a custom redelivery handler" must {
"handle thrown exceptions by redelivering the initial message" in {
service.awaitEndpointActivation(1) {
actorOf[RedeliveringConsumer].start
} must be (true)
mandatoryTemplate.requestBody("direct:redelivery-test", "hello") must equal ("accepted: hello")
}
}
}
}
object ConsumerScalaTest {
trait BlockingConsumer extends Consumer { self: Actor =>
override def blocking = true
}
class TestConsumer(uri: String) extends Actor with Consumer {
def endpointUri = uri
protected def receive = {
case msg: Message => self.reply("received %s" format msg.body)
}
}
class TestBlocker(uri: String) extends Actor with BlockingConsumer {
self.timeout = 1000
def endpointUri = uri
protected def receive = {
case msg: Message => { /* do not reply */ }
}
}
class ErrorHandlingConsumer extends Actor with BlockingConsumer {
def endpointUri = "direct:error-handler-test"
onRouteDefinition {rd: RouteDefinition =>
rd.onException(classOf[Exception]).handled(true).transform(Builder.exceptionMessage).end
}
protected def receive = {
case msg: Message => throw new Exception("error: %s" format msg.body)
}
}
class RedeliveringConsumer extends Actor with BlockingConsumer {
def endpointUri = "direct:redelivery-test"
onRouteDefinition {rd: RouteDefinition =>
rd.onException(classOf[Exception]).maximumRedeliveries(1).end
}
//
// first message to this actor is not valid and will be rejected
//
var valid = false
protected def receive = {
case msg: Message => try {
respondTo(msg)
} finally {
valid = true
}
}
private def respondTo(msg: Message) =
if (valid) self.reply("accepted: %s" format msg.body)
else throw new Exception("rejected: %s" format msg.body)
}
trait TestTypedConsumer {
@consume("direct:publish-test-3")
def foo(s: String): String
def bar(s: String): String
}
class TestTypedConsumerImpl extends TypedActor with TestTypedConsumer {
def foo(s: String) = "foo: %s" format s
@consume("direct:publish-test-4")
def bar(s: String) = "bar: %s" format s
}
}

View file

@ -1,5 +0,0 @@
package akka.camel
import org.scalatest.junit.JUnitSuite
class MessageJavaTest extends MessageJavaTestBase with JUnitSuite

View file

@ -1,83 +0,0 @@
package akka.camel
import java.io.InputStream
import org.apache.camel.NoTypeConversionAvailableException
import org.junit.Assert._
import org.junit.Test
import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitSuite
class MessageScalaTest extends JUnitSuite with BeforeAndAfterAll {
override protected def beforeAll = CamelContextManager.init
@Test def shouldConvertDoubleBodyToString = {
assertEquals("1.4", Message(1.4).bodyAs[String])
}
@Test def shouldThrowExceptionWhenConvertingDoubleBodyToInputStream {
intercept[NoTypeConversionAvailableException] {
Message(1.4).bodyAs[InputStream]
}
}
@Test def shouldReturnDoubleHeader = {
val message = Message("test" , Map("test" -> 1.4))
assertEquals(1.4, message.header("test"))
}
@Test def shouldConvertDoubleHeaderToString = {
val message = Message("test" , Map("test" -> 1.4))
assertEquals("1.4", message.headerAs[String]("test"))
}
@Test def shouldReturnSubsetOfHeaders = {
val message = Message("test" , Map("A" -> "1", "B" -> "2"))
assertEquals(Map("B" -> "2"), message.headers(Set("B")))
}
@Test def shouldTransformBodyAndPreserveHeaders = {
assertEquals(
Message("ab", Map("A" -> "1")),
Message("a" , Map("A" -> "1")).transformBody((body: String) => body + "b"))
}
@Test def shouldConvertBodyAndPreserveHeaders = {
assertEquals(
Message("1.4", Map("A" -> "1")),
Message(1.4 , Map("A" -> "1")).setBodyAs[String])
}
@Test def shouldSetBodyAndPreserveHeaders = {
assertEquals(
Message("test2" , Map("A" -> "1")),
Message("test1" , Map("A" -> "1")).setBody("test2"))
}
@Test def shouldSetHeadersAndPreserveBody = {
assertEquals(
Message("test1" , Map("C" -> "3")),
Message("test1" , Map("A" -> "1")).setHeaders(Map("C" -> "3")))
}
@Test def shouldAddHeaderAndPreserveBodyAndHeaders = {
assertEquals(
Message("test1" , Map("A" -> "1", "B" -> "2")),
Message("test1" , Map("A" -> "1")).addHeader("B" -> "2"))
}
@Test def shouldAddHeadersAndPreserveBodyAndHeaders = {
assertEquals(
Message("test1" , Map("A" -> "1", "B" -> "2")),
Message("test1" , Map("A" -> "1")).addHeaders(Map("B" -> "2")))
}
@Test def shouldRemoveHeadersAndPreserveBodyAndRemainingHeaders = {
assertEquals(
Message("test1" , Map("A" -> "1")),
Message("test1" , Map("A" -> "1", "B" -> "2")).removeHeader("B"))
}
}

View file

@ -1,301 +0,0 @@
package akka.camel
import org.apache.camel.{Exchange, Processor}
import org.apache.camel.builder.RouteBuilder
import org.apache.camel.component.mock.MockEndpoint
import org.scalatest.{GivenWhenThen, BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
import akka.actor.Actor._
import akka.actor.{ActorRef, Actor, ActorRegistry}
class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen {
import ProducerFeatureTest._
override protected def beforeAll = {
ActorRegistry.shutdownAll
CamelContextManager.init
CamelContextManager.mandatoryContext.addRoutes(new TestRoute)
CamelContextManager.start
}
override protected def afterAll = {
CamelContextManager.stop
ActorRegistry.shutdownAll
}
override protected def afterEach = {
mockEndpoint.reset
}
feature("Produce a message to a sync Camel route") {
scenario("produce message and receive normal response") {
given("a registered two-way producer")
val producer = actorOf(new TestProducer("direct:producer-test-2", true))
producer.start
when("a test message is sent to the producer with !!")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
val result = producer !! message
then("a normal response should have been returned by the producer")
val expected = Message("received TEST", Map(Message.MessageExchangeId -> "123"))
assert(result === Some(expected))
}
scenario("produce message and receive failure response") {
given("a registered two-way producer")
val producer = actorOf(new TestProducer("direct:producer-test-2"))
producer.start
when("a test message causing an exception is sent to the producer with !!")
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
val result = (producer !! message).as[Failure]
then("a failure response should have been returned by the producer")
val expectedFailureText = result.get.cause.getMessage
val expectedHeaders = result.get.headers
assert(expectedFailureText === "failure")
assert(expectedHeaders === Map(Message.MessageExchangeId -> "123"))
}
scenario("produce message oneway") {
given("a registered one-way producer")
val producer = actorOf(new TestProducer("direct:producer-test-1", true) with Oneway)
producer.start
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("TEST")
producer ! Message("test")
then("the test message should have been sent to mock:mock")
mockEndpoint.assertIsSatisfied
}
scenario("produce message twoway without sender reference") {
given("a registered two-way producer")
val producer = actorOf(new TestProducer("direct:producer-test-1"))
producer.start
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("test")
producer ! Message("test")
then("there should be only a warning that there's no sender reference")
mockEndpoint.assertIsSatisfied
}
}
feature("Produce a message to an async Camel route") {
scenario("produce message and receive normal response") {
given("a registered two-way producer")
val producer = actorOf(new TestProducer("direct:producer-test-3"))
producer.start
when("a test message is sent to the producer with !!")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
val result = producer !! message
then("a normal response should have been returned by the producer")
val expected = Message("received test", Map(Message.MessageExchangeId -> "123"))
assert(result === Some(expected))
}
scenario("produce message and receive failure response") {
given("a registered two-way producer")
val producer = actorOf(new TestProducer("direct:producer-test-3"))
producer.start
when("a test message causing an exception is sent to the producer with !!")
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
val result = (producer !! message).as[Failure]
then("a failure response should have been returned by the producer")
val expectedFailureText = result.get.cause.getMessage
val expectedHeaders = result.get.headers
assert(expectedFailureText === "failure")
assert(expectedHeaders === Map(Message.MessageExchangeId -> "123"))
}
}
feature("Produce a message to a sync Camel route and then forward the response") {
scenario("produce message, forward normal response to a replying target actor and receive response") {
given("a registered two-way producer configured with a forward target")
val target = actorOf[ReplyingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
when("a test message is sent to the producer with !!")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
val result = producer !! message
then("a normal response should have been returned by the forward target")
val expected = Message("received test", Map(Message.MessageExchangeId -> "123", "test" -> "result"))
assert(result === Some(expected))
}
scenario("produce message, forward failure response to a replying target actor and receive response") {
given("a registered two-way producer configured with a forward target")
val target = actorOf[ReplyingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
when("a test message causing an exception is sent to the producer with !!")
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
val result = (producer !! message).as[Failure]
then("a failure response should have been returned by the forward target")
val expectedFailureText = result.get.cause.getMessage
val expectedHeaders = result.get.headers
assert(expectedFailureText === "failure")
assert(expectedHeaders === Map(Message.MessageExchangeId -> "123", "test" -> "failure"))
}
scenario("produce message, forward normal response to a producing target actor and produce response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val target = actorOf[ProducingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("received test")
val result = producer.!(Message("test"))(Some(producer))
then("a normal response should have been produced by the forward target")
mockEndpoint.assertIsSatisfied
}
scenario("produce message, forward failure response to a producing target actor and produce response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val target = actorOf[ProducingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
when("a test message causing an exception is sent to the producer with !")
mockEndpoint.expectedMessageCount(1)
mockEndpoint.message(0).body().isInstanceOf(classOf[Failure])
val result = producer.!(Message("fail"))(Some(producer))
then("a failure response should have been produced by the forward target")
mockEndpoint.assertIsSatisfied
}
}
feature("Produce a message to an async Camel route and then forward the response") {
scenario("produce message, forward normal response to a replying target actor and receive response") {
given("a registered two-way producer configured with a forward target")
val target = actorOf[ReplyingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
when("a test message is sent to the producer with !!")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
val result = producer !! message
then("a normal response should have been returned by the forward target")
val expected = Message("received test", Map(Message.MessageExchangeId -> "123", "test" -> "result"))
assert(result === Some(expected))
}
scenario("produce message, forward failure response to a replying target actor and receive response") {
given("a registered two-way producer configured with a forward target")
val target = actorOf[ReplyingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
when("a test message causing an exception is sent to the producer with !!")
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
val result = (producer !! message).as[Failure]
then("a failure response should have been returned by the forward target")
val expectedFailureText = result.get.cause.getMessage
val expectedHeaders = result.get.headers
assert(expectedFailureText === "failure")
assert(expectedHeaders === Map(Message.MessageExchangeId -> "123", "test" -> "failure"))
}
scenario("produce message, forward normal response to a producing target actor and produce response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val target = actorOf[ProducingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("received test")
val result = producer.!(Message("test"))(Some(producer))
then("a normal response should have been produced by the forward target")
mockEndpoint.assertIsSatisfied
}
scenario("produce message, forward failure response to a producing target actor and produce response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val target = actorOf[ProducingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
when("a test message causing an exception is sent to the producer with !")
mockEndpoint.expectedMessageCount(1)
mockEndpoint.message(0).body().isInstanceOf(classOf[Failure])
val result = producer.!(Message("fail"))(Some(producer))
then("a failure response should have been produced by the forward target")
mockEndpoint.assertIsSatisfied
}
}
private def mockEndpoint = CamelContextManager.mandatoryContext.getEndpoint("mock:mock", classOf[MockEndpoint])
}
object ProducerFeatureTest {
class TestProducer(uri: String, upper: Boolean = false) extends Actor with Producer {
def endpointUri = uri
override protected def receiveBeforeProduce = {
case msg: Message => if (upper) msg.transformBody { body: String => body.toUpperCase } else msg
}
}
class TestForwarder(uri: String, target: ActorRef) extends Actor with Producer {
def endpointUri = uri
override protected def receiveAfterProduce = {
case msg => target forward msg
}
}
class TestResponder extends Actor {
protected def receive = {
case msg: Message => msg.body match {
case "fail" => self.reply(Failure(new Exception("failure"), msg.headers))
case _ => self.reply(msg.transformBody { body: String => "received %s" format body })
}
}
}
class ReplyingForwardTarget extends Actor {
protected def receive = {
case msg: Message =>
self.reply(msg.addHeader("test" -> "result"))
case msg: Failure =>
self.reply(Failure(msg.cause, msg.headers + ("test" -> "failure")))
}
}
class ProducingForwardTarget extends Actor with Producer with Oneway {
def endpointUri = "direct:forward-test-1"
}
class TestRoute extends RouteBuilder {
val responder = actorOf[TestResponder].start
def configure {
from("direct:forward-test-1").to("mock:mock")
// for one-way messaging tests
from("direct:producer-test-1").to("mock:mock")
// for two-way messaging tests (async)
from("direct:producer-test-3").to("actor:uuid:%s" format responder.uuid)
// for two-way messaging tests (sync)
from("direct:producer-test-2").process(new Processor() {
def process(exchange: Exchange) = {
exchange.getIn.getBody match {
case "fail" => throw new Exception("failure")
case body => exchange.getOut.setBody("received %s" format body)
}
}
})
}
}
}

View file

@ -1,103 +0,0 @@
package akka.camel
import java.util.concurrent.{CountDownLatch, TimeUnit}
import org.junit.{Before, After, Test}
import org.scalatest.junit.JUnitSuite
import akka.actor._
import akka.actor.Actor._
import akka.camel.support.{SetExpectedMessageCount => SetExpectedTestMessageCount, _}
class PublishRequestorTest extends JUnitSuite {
import PublishRequestorTest._
var publisher: ActorRef = _
var requestor: ActorRef = _
var consumer: ActorRef = _
val ascendingMethodName = (r1: ConsumerMethodRegistered, r2: ConsumerMethodRegistered) =>
r1.method.getName < r2.method.getName
@Before def setUp: Unit = {
publisher = actorOf[PublisherMock].start
requestor = actorOf[PublishRequestor].start
requestor ! PublishRequestorInit(publisher)
consumer = actorOf(new Actor with Consumer {
def endpointUri = "mock:test"
protected def receive = null
}).start
}
@After def tearDown = {
AspectInitRegistry.removeListener(requestor);
ActorRegistry.shutdownAll
}
@Test def shouldReceiveOneConsumerMethodRegisteredEvent = {
AspectInitRegistry.addListener(requestor)
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
val obj = TypedActor.newInstance(classOf[SampleTypedSingleConsumer], classOf[SampleTypedSingleConsumerImpl])
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val event = (publisher !! GetRetainedMessage).as[ConsumerMethodRegistered].get
assert(event.endpointUri === "direct:foo")
assert(event.typedActor === obj)
assert(event.methodName === "foo")
}
@Test def shouldReceiveOneConsumerMethodUnregisteredEvent = {
val obj = TypedActor.newInstance(classOf[SampleTypedSingleConsumer], classOf[SampleTypedSingleConsumerImpl])
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
AspectInitRegistry.addListener(requestor)
TypedActor.stop(obj)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val event = (publisher !! GetRetainedMessage).as[ConsumerMethodUnregistered].get
assert(event.endpointUri === "direct:foo")
assert(event.typedActor === obj)
assert(event.methodName === "foo")
}
@Test def shouldReceiveThreeConsumerMethodRegisteredEvents = {
AspectInitRegistry.addListener(requestor)
val latch = (publisher !! SetExpectedTestMessageCount(3)).as[CountDownLatch].get
val obj = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val request = GetRetainedMessages(_.isInstanceOf[ConsumerMethodRegistered])
val events = (publisher !! request).as[List[ConsumerMethodRegistered]].get
assert(events.map(_.method.getName).sortWith(_ < _) === List("m2", "m3", "m4"))
}
@Test def shouldReceiveThreeConsumerMethodUnregisteredEvents = {
val obj = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
val latch = (publisher !! SetExpectedTestMessageCount(3)).as[CountDownLatch].get
AspectInitRegistry.addListener(requestor)
TypedActor.stop(obj)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val request = GetRetainedMessages(_.isInstanceOf[ConsumerMethodUnregistered])
val events = (publisher !! request).as[List[ConsumerMethodUnregistered]].get
assert(events.map(_.method.getName).sortWith(_ < _) === List("m2", "m3", "m4"))
}
@Test def shouldReceiveOneConsumerRegisteredEvent = {
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
requestor ! ActorRegistered(consumer)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
assert((publisher !! GetRetainedMessage) ===
Some(ConsumerActorRegistered(consumer, consumer.actor.asInstanceOf[Consumer])))
}
@Test def shouldReceiveOneConsumerUnregisteredEvent = {
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
requestor ! ActorUnregistered(consumer)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
assert((publisher !! GetRetainedMessage) ===
Some(ConsumerActorUnregistered(consumer, consumer.actor.asInstanceOf[Consumer])))
}
}
object PublishRequestorTest {
class PublisherMock extends TestActor with Retain with Countdown {
def handler = retain andThen countdown
}
}

View file

@ -1,101 +0,0 @@
package akka.camel
import java.util.concurrent.{CountDownLatch, TimeUnit}
import org.scalatest.{GivenWhenThen, BeforeAndAfterAll, FeatureSpec}
import akka.actor._
import akka.actor.Actor._
import akka.remote.{RemoteClient, RemoteServer}
/**
* @author Martin Krasser
*/
class RemoteConsumerTest extends FeatureSpec with BeforeAndAfterAll with GivenWhenThen {
import CamelServiceManager._
import RemoteConsumerTest._
var server: RemoteServer = _
override protected def beforeAll = {
ActorRegistry.shutdownAll
startCamelService
server = new RemoteServer()
server.start(host, port)
Thread.sleep(1000)
}
override protected def afterAll = {
server.shutdown
stopCamelService
RemoteClient.shutdownAll
ActorRegistry.shutdownAll
Thread.sleep(1000)
}
feature("Publish consumer on remote node") {
scenario("access published remote consumer") {
given("a client-initiated remote consumer")
val consumer = actorOf[RemoteConsumer].start
when("remote consumer publication is triggered")
assert(mandatoryService.awaitEndpointActivation(1) {
consumer !! "init"
})
then("the published consumer is accessible via its endpoint URI")
val response = CamelContextManager.mandatoryTemplate.requestBody("direct:remote-consumer", "test")
assert(response === "remote actor: test")
}
}
feature("Publish typed consumer on remote node") {
scenario("access published remote consumer method") {
given("a client-initiated remote typed consumer")
val consumer = TypedActor.newRemoteInstance(classOf[SampleRemoteTypedConsumer], classOf[SampleRemoteTypedConsumerImpl], host, port)
when("remote typed consumer publication is triggered")
assert(mandatoryService.awaitEndpointActivation(1) {
consumer.foo("init")
})
then("the published method is accessible via its endpoint URI")
val response = CamelContextManager.mandatoryTemplate.requestBody("direct:remote-typed-consumer", "test")
assert(response === "remote typed actor: test")
}
}
feature("Publish untyped consumer on remote node") {
scenario("access published remote untyped consumer") {
given("a client-initiated remote untyped consumer")
val consumer = UntypedActor.actorOf(classOf[SampleRemoteUntypedConsumer]).start
when("remote untyped consumer publication is triggered")
assert(mandatoryService.awaitEndpointActivation(1) {
consumer.sendRequestReply(Message("init", Map("test" -> "init")))
})
then("the published untyped consumer is accessible via its endpoint URI")
val response = CamelContextManager.mandatoryTemplate.requestBodyAndHeader("direct:remote-untyped-consumer", "a", "test", "b")
assert(response === "a b")
}
}
}
object RemoteConsumerTest {
val host = "localhost"
val port = 7774
class RemoteConsumer extends RemoteActor(host, port) with Consumer {
def endpointUri = "direct:remote-consumer"
protected def receive = {
case "init" => self.reply("done")
case m: Message => self.reply("remote actor: %s" format m.body)
}
}
}

View file

@ -1,98 +0,0 @@
package akka.camel
import org.apache.camel.{Exchange, Processor}
import org.apache.camel.builder.RouteBuilder
import org.apache.camel.component.mock.MockEndpoint
import org.scalatest.{GivenWhenThen, BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
import akka.actor.UntypedActor._
import akka.actor.ActorRegistry
class UntypedProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen {
import UntypedProducerFeatureTest._
override protected def beforeAll = {
ActorRegistry.shutdownAll
CamelContextManager.init
CamelContextManager.mandatoryContext.addRoutes(new TestRoute)
CamelContextManager.start
}
override protected def afterAll = {
CamelContextManager.stop
ActorRegistry.shutdownAll
}
override protected def afterEach = {
mockEndpoint.reset
}
feature("Produce a message to a sync Camel route") {
scenario("produce message and receive normal response") {
given("a registered two-way producer")
val producer = actorOf(classOf[SampleUntypedReplyingProducer])
producer.start
when("a test message is sent to the producer with !!")
val message = Message("test", Map(Message.MessageExchangeId -> "123"))
val result = producer.sendRequestReply(message)
then("a normal response should have been returned by the producer")
val expected = Message("received test", Map(Message.MessageExchangeId -> "123"))
assert(result === expected)
}
scenario("produce message and receive failure response") {
given("a registered two-way producer")
val producer = actorOf(classOf[SampleUntypedReplyingProducer])
producer.start
when("a test message causing an exception is sent to the producer with !!")
val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
val result = producer.sendRequestReply(message).asInstanceOf[Failure]
then("a failure response should have been returned by the producer")
val expectedFailureText = result.cause.getMessage
val expectedHeaders = result.headers
assert(expectedFailureText === "failure")
assert(expectedHeaders === Map(Message.MessageExchangeId -> "123"))
}
}
feature("Produce a message to a sync Camel route and then forward the response") {
scenario("produce message and send normal response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val producer = actorOf(classOf[SampleUntypedForwardingProducer])
producer.start
when("a test message is sent to the producer with !")
mockEndpoint.expectedBodiesReceived("received test")
val result = producer.sendOneWay(Message("test"), producer)
then("a normal response should have been sent")
mockEndpoint.assertIsSatisfied
}
}
private def mockEndpoint = CamelContextManager.mandatoryContext.getEndpoint("mock:mock", classOf[MockEndpoint])
}
object UntypedProducerFeatureTest {
class TestRoute extends RouteBuilder {
def configure {
from("direct:forward-test-1").to("mock:mock")
from("direct:producer-test-1").process(new Processor() {
def process(exchange: Exchange) = {
exchange.getIn.getBody match {
case "fail" => throw new Exception("failure")
case body => exchange.getOut.setBody("received %s" format body)
}
}
})
}
}
}

View file

@ -1,130 +0,0 @@
package akka.camel.component
import java.util.concurrent.{TimeUnit, CountDownLatch}
import org.apache.camel.RuntimeCamelException
import org.apache.camel.builder.RouteBuilder
import org.apache.camel.component.mock.MockEndpoint
import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
import akka.actor.Actor._
import akka.actor.{ActorRegistry, Actor}
import akka.camel.{Failure, Message, CamelContextManager}
import akka.camel.support._
class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach {
import ActorComponentFeatureTest._
override protected def beforeAll = {
ActorRegistry.shutdownAll
CamelContextManager.init
CamelContextManager.mandatoryContext.addRoutes(new TestRoute)
CamelContextManager.start
}
override protected def afterAll = CamelContextManager.stop
override protected def afterEach = {
ActorRegistry.shutdownAll
mockEndpoint.reset
}
feature("Communicate with an actor via an actor:uuid endpoint") {
import CamelContextManager.mandatoryTemplate
scenario("one-way communication") {
val actor = actorOf[Tester1].start
val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
mandatoryTemplate.sendBody("actor:uuid:%s" format actor.uuid, "Martin")
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val reply = (actor !! GetRetainedMessage).get.asInstanceOf[Message]
assert(reply.body === "Martin")
}
scenario("two-way communication") {
val actor = actorOf[Tester2].start
assert(mandatoryTemplate.requestBody("actor:uuid:%s" format actor.uuid, "Martin") === "Hello Martin")
}
scenario("two-way communication with timeout") {
val actor = actorOf[Tester3].start
intercept[RuntimeCamelException] {
mandatoryTemplate.requestBody("actor:uuid:%s?blocking=true" format actor.uuid, "Martin")
}
}
scenario("two-way communication via a custom route with failure response") {
mockEndpoint.expectedBodiesReceived("whatever")
mandatoryTemplate.requestBody("direct:failure-test-1", "whatever")
mockEndpoint.assertIsSatisfied
}
scenario("two-way communication via a custom route with exception") {
mockEndpoint.expectedBodiesReceived("whatever")
mandatoryTemplate.requestBody("direct:failure-test-2", "whatever")
mockEndpoint.assertIsSatisfied
}
}
feature("Communicate with an actor via an actor:id endpoint") {
import CamelContextManager.mandatoryTemplate
scenario("one-way communication") {
val actor = actorOf[Tester1].start
val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
mandatoryTemplate.sendBody("actor:%s" format actor.id, "Martin")
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val reply = (actor !! GetRetainedMessage).get.asInstanceOf[Message]
assert(reply.body === "Martin")
}
scenario("two-way communication") {
val actor = actorOf[Tester2].start
assert(mandatoryTemplate.requestBody("actor:%s" format actor.id, "Martin") === "Hello Martin")
}
scenario("two-way communication via a custom route") {
val actor = actorOf[CustomIdActor].start
assert(mandatoryTemplate.requestBody("direct:custom-id-test-1", "Martin") === "Received Martin")
assert(mandatoryTemplate.requestBody("direct:custom-id-test-2", "Martin") === "Received Martin")
}
}
private def mockEndpoint = CamelContextManager.mandatoryContext.getEndpoint("mock:mock", classOf[MockEndpoint])
}
object ActorComponentFeatureTest {
class CustomIdActor extends Actor {
self.id = "custom-id"
protected def receive = {
case msg: Message => self.reply("Received %s" format msg.body)
}
}
class FailWithMessage extends Actor {
protected def receive = {
case msg: Message => self.reply(Failure(new Exception("test")))
}
}
class FailWithException extends Actor {
protected def receive = {
case msg: Message => throw new Exception("test")
}
}
class TestRoute extends RouteBuilder {
val failWithMessage = actorOf[FailWithMessage].start
val failWithException = actorOf[FailWithException].start
def configure {
from("direct:custom-id-test-1").to("actor:custom-id")
from("direct:custom-id-test-2").to("actor:id:custom-id")
from("direct:failure-test-1")
.onException(classOf[Exception]).to("mock:mock").handled(true).end
.to("actor:uuid:%s" format failWithMessage.uuid)
from("direct:failure-test-2")
.onException(classOf[Exception]).to("mock:mock").handled(true).end
.to("actor:uuid:%s?blocking=true" format failWithException.uuid)
}
}
}

View file

@ -1,79 +0,0 @@
package akka.camel.component
import org.apache.camel.{Endpoint, AsyncProcessor}
import org.apache.camel.impl.DefaultCamelContext
import org.junit._
import org.scalatest.junit.JUnitSuite
import akka.actor.uuidFrom
class ActorComponentTest extends JUnitSuite {
val component: ActorComponent = ActorComponentTest.actorComponent
def testUUID = "93da8c80-c3fd-11df-abed-60334b120057"
@Test def shouldCreateEndpointWithIdDefined = {
val ep1: ActorEndpoint = component.createEndpoint("actor:abc").asInstanceOf[ActorEndpoint]
val ep2: ActorEndpoint = component.createEndpoint("actor:id:abc").asInstanceOf[ActorEndpoint]
assert(ep1.idValue === Some("abc"))
assert(ep2.idValue === Some("abc"))
assert(ep1.idType === "id")
assert(ep2.idType === "id")
assert(!ep1.blocking)
assert(!ep2.blocking)
}
@Test def shouldCreateEndpointWithIdTemplate = {
val ep: ActorEndpoint = component.createEndpoint("actor:id:").asInstanceOf[ActorEndpoint]
assert(ep.idValue === None)
assert(ep.idType === "id")
assert(!ep.blocking)
}
@Test def shouldCreateEndpointWithIdTemplateAndBlockingSet = {
val ep: ActorEndpoint = component.createEndpoint("actor:id:?blocking=true").asInstanceOf[ActorEndpoint]
assert(ep.idValue === None)
assert(ep.idType === "id")
assert(ep.blocking)
}
@Test def shouldCreateEndpointWithUuidDefined = {
val ep: ActorEndpoint = component.createEndpoint("actor:uuid:%s" format testUUID).asInstanceOf[ActorEndpoint]
assert(ep.idValue === Some(testUUID))
assert(ep.idType === "uuid")
assert(!ep.blocking)
}
@Test def shouldCreateEndpointWithUuidTemplate = {
val ep: ActorEndpoint = component.createEndpoint("actor:uuid:").asInstanceOf[ActorEndpoint]
assert(ep.idValue === None)
assert(ep.idType === "uuid")
assert(!ep.blocking)
}
@Test def shouldCreateEndpointWithUuidTemplateandBlockingSet = {
val ep: ActorEndpoint = component.createEndpoint("actor:uuid:?blocking=true").asInstanceOf[ActorEndpoint]
assert(ep.idValue === None)
assert(ep.idType === "uuid")
assert(ep.blocking)
}
@Test def shouldCreateEndpointWithBlockingSet = {
val ep: ActorEndpoint = component.createEndpoint("actor:uuid:%s?blocking=true" format testUUID).asInstanceOf[ActorEndpoint]
assert(ep.idValue === Some(testUUID))
assert(ep.idType === "uuid")
assert(ep.blocking)
}
}
object ActorComponentTest {
def actorComponent = {
val component = new ActorComponent
component.setCamelContext(new DefaultCamelContext)
component
}
def actorEndpoint(uri:String) = actorComponent.createEndpoint(uri)
def actorProducer(endpoint: Endpoint) = endpoint.createProducer
def actorAsyncProducer(endpoint: Endpoint) = endpoint.createProducer.asInstanceOf[AsyncProcessor]
}

View file

@ -1,230 +0,0 @@
package akka.camel.component
import ActorComponentTest._
import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit}
import org.apache.camel.{AsyncCallback, ExchangePattern}
import org.junit.{After, Test}
import org.scalatest.junit.JUnitSuite
import org.scalatest.BeforeAndAfterAll
import akka.actor.Actor._
import akka.actor.ActorRegistry
import akka.camel.{Failure, Message}
import akka.camel.support._
class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
import ActorProducerTest._
@After def tearDown = ActorRegistry.shutdownAll
@Test def shouldSendMessageToActorWithSyncProcessor = {
val actor = actorOf[Tester1].start
val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOnly)
exchange.getIn.setBody("Martin")
exchange.getIn.setHeader("k1", "v1")
actorProducer(endpoint).process(exchange)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val reply = (actor !! GetRetainedMessage).get.asInstanceOf[Message]
assert(reply.body === "Martin")
assert(reply.headers === Map(Message.MessageExchangeId -> exchange.getExchangeId, "k1" -> "v1"))
}
@Test def shouldSendMessageToActorWithAsyncProcessor = {
val actor = actorOf[Tester1].start
val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOnly)
exchange.getIn.setBody("Martin")
exchange.getIn.setHeader("k1", "v1")
actorAsyncProducer(endpoint).process(exchange, expectSyncCompletion)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
val reply = (actor !! GetRetainedMessage).get.asInstanceOf[Message]
assert(reply.body === "Martin")
assert(reply.headers === Map(Message.MessageExchangeId -> exchange.getExchangeId, "k1" -> "v1"))
}
@Test def shouldSendMessageToActorAndReceiveResponseWithSyncProcessor = {
val actor = actorOf(new Tester2 {
override def response(msg: Message) = Message(super.response(msg), Map("k2" -> "v2"))
}).start
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)
exchange.getIn.setBody("Martin")
exchange.getIn.setHeader("k1", "v1")
actorProducer(endpoint).process(exchange)
assert(exchange.getOut.getBody === "Hello Martin")
assert(exchange.getOut.getHeader("k2") === "v2")
}
@Test def shouldSendMessageToActorAndReceiveResponseWithAsyncProcessor = {
val actor = actorOf(new Tester2 {
override def response(msg: Message) = Message(super.response(msg), Map("k2" -> "v2"))
}).start
val completion = expectAsyncCompletion
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)
exchange.getIn.setBody("Martin")
exchange.getIn.setHeader("k1", "v1")
actorAsyncProducer(endpoint).process(exchange, completion)
assert(completion.latch.await(5000, TimeUnit.MILLISECONDS))
assert(exchange.getOut.getBody === "Hello Martin")
assert(exchange.getOut.getHeader("k2") === "v2")
}
@Test def shouldSendMessageToActorAndReceiveFailureWithAsyncProcessor = {
val actor = actorOf(new Tester2 {
override def response(msg: Message) = Failure(new Exception("testmsg"), Map("k3" -> "v3"))
}).start
val completion = expectAsyncCompletion
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)
exchange.getIn.setBody("Martin")
exchange.getIn.setHeader("k1", "v1")
actorAsyncProducer(endpoint).process(exchange, completion)
assert(completion.latch.await(5000, TimeUnit.MILLISECONDS))
assert(exchange.getException.getMessage === "testmsg")
assert(exchange.getOut.getBody === null)
assert(exchange.getOut.getHeader("k3") === null) // headers from failure message are currently ignored
}
@Test def shouldDynamicallyRouteMessageToActorWithDefaultId = {
val actor1 = actorOf[Tester1]
val actor2 = actorOf[Tester1]
actor1.id = "x"
actor2.id = "y"
actor1.start
actor2.start
val latch1 = (actor1 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val latch2 = (actor2 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:id:%s" format actor1.id)
val exchange1 = endpoint.createExchange(ExchangePattern.InOnly)
val exchange2 = endpoint.createExchange(ExchangePattern.InOnly)
exchange1.getIn.setBody("Test1")
exchange2.getIn.setBody("Test2")
exchange2.getIn.setHeader(ActorComponent.ActorIdentifier, actor2.id)
actorProducer(endpoint).process(exchange1)
actorProducer(endpoint).process(exchange2)
assert(latch1.await(5, TimeUnit.SECONDS))
assert(latch2.await(5, TimeUnit.SECONDS))
val reply1 = (actor1 !! GetRetainedMessage).get.asInstanceOf[Message]
val reply2 = (actor2 !! GetRetainedMessage).get.asInstanceOf[Message]
assert(reply1.body === "Test1")
assert(reply2.body === "Test2")
}
@Test def shouldDynamicallyRouteMessageToActorWithoutDefaultId = {
val actor1 = actorOf[Tester1]
val actor2 = actorOf[Tester1]
actor1.id = "x"
actor2.id = "y"
actor1.start
actor2.start
val latch1 = (actor1 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val latch2 = (actor2 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:id:")
val exchange1 = endpoint.createExchange(ExchangePattern.InOnly)
val exchange2 = endpoint.createExchange(ExchangePattern.InOnly)
exchange1.getIn.setBody("Test1")
exchange2.getIn.setBody("Test2")
exchange1.getIn.setHeader(ActorComponent.ActorIdentifier, actor1.id)
exchange2.getIn.setHeader(ActorComponent.ActorIdentifier, actor2.id)
actorProducer(endpoint).process(exchange1)
actorProducer(endpoint).process(exchange2)
assert(latch1.await(5, TimeUnit.SECONDS))
assert(latch2.await(5, TimeUnit.SECONDS))
val reply1 = (actor1 !! GetRetainedMessage).get.asInstanceOf[Message]
val reply2 = (actor2 !! GetRetainedMessage).get.asInstanceOf[Message]
assert(reply1.body === "Test1")
assert(reply2.body === "Test2")
}
@Test def shouldDynamicallyRouteMessageToActorWithDefaultUuid = {
val actor1 = actorOf[Tester1].start
val actor2 = actorOf[Tester1].start
val latch1 = (actor1 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val latch2 = (actor2 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:uuid:%s" format actor1.uuid)
val exchange1 = endpoint.createExchange(ExchangePattern.InOnly)
val exchange2 = endpoint.createExchange(ExchangePattern.InOnly)
exchange1.getIn.setBody("Test1")
exchange2.getIn.setBody("Test2")
exchange2.getIn.setHeader(ActorComponent.ActorIdentifier, actor2.uuid.toString)
actorProducer(endpoint).process(exchange1)
actorProducer(endpoint).process(exchange2)
assert(latch1.await(5, TimeUnit.SECONDS))
assert(latch2.await(5, TimeUnit.SECONDS))
val reply1 = (actor1 !! GetRetainedMessage).get.asInstanceOf[Message]
val reply2 = (actor2 !! GetRetainedMessage).get.asInstanceOf[Message]
assert(reply1.body === "Test1")
assert(reply2.body === "Test2")
}
@Test def shouldDynamicallyRouteMessageToActorWithoutDefaultUuid = {
val actor1 = actorOf[Tester1].start
val actor2 = actorOf[Tester1].start
val latch1 = (actor1 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val latch2 = (actor2 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:uuid:")
val exchange1 = endpoint.createExchange(ExchangePattern.InOnly)
val exchange2 = endpoint.createExchange(ExchangePattern.InOnly)
exchange1.getIn.setBody("Test1")
exchange2.getIn.setBody("Test2")
exchange1.getIn.setHeader(ActorComponent.ActorIdentifier, actor1.uuid)
exchange2.getIn.setHeader(ActorComponent.ActorIdentifier, actor2.uuid.toString)
actorProducer(endpoint).process(exchange1)
actorProducer(endpoint).process(exchange2)
assert(latch1.await(5, TimeUnit.SECONDS))
assert(latch2.await(5, TimeUnit.SECONDS))
val reply1 = (actor1 !! GetRetainedMessage).get.asInstanceOf[Message]
val reply2 = (actor2 !! GetRetainedMessage).get.asInstanceOf[Message]
assert(reply1.body === "Test1")
assert(reply2.body === "Test2")
}
@Test def shouldThrowExceptionWhenIdNotSet: Unit = {
val actor = actorOf[Tester1].start
val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:id:")
intercept[ActorIdentifierNotSetException] {
actorProducer(endpoint).process(endpoint.createExchange(ExchangePattern.InOnly))
}
}
@Test def shouldThrowExceptionWhenUuidNotSet: Unit = {
val actor = actorOf[Tester1].start
val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
val endpoint = actorEndpoint("actor:uuid:")
intercept[ActorIdentifierNotSetException] {
actorProducer(endpoint).process(endpoint.createExchange(ExchangePattern.InOnly))
}
}
@Test def shouldSendMessageToActorAndTimeout(): Unit = {
val actor = actorOf[Tester3].start
val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)
exchange.getIn.setBody("Martin")
intercept[TimeoutException] {
endpoint.createProducer.process(exchange)
}
}
}
object ActorProducerTest {
def expectSyncCompletion = new AsyncCallback {
def done(doneSync: Boolean) = assert(doneSync)
}
def expectAsyncCompletion = new AsyncCallback {
val latch = new CountDownLatch(1);
def done(doneSync: Boolean) = {
assert(!doneSync)
latch.countDown
}
}
}

View file

@ -1,108 +0,0 @@
package akka.camel.component
import org.apache.camel._
import org.apache.camel.builder.RouteBuilder
import org.apache.camel.impl.{DefaultCamelContext, SimpleRegistry}
import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
import akka.actor.{ActorRegistry, TypedActor}
import akka.camel._
/**
* @author Martin Krasser
*/
class TypedActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach {
import TypedActorComponentFeatureTest._
import CamelContextManager.mandatoryTemplate
override protected def beforeAll = {
val typedActor = TypedActor.newInstance(classOf[SampleTypedActor], classOf[SampleTypedActorImpl]) // not a consumer
val typedConsumer = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
val registry = new SimpleRegistry
// external registration
registry.put("ta", typedActor)
CamelContextManager.init(new DefaultCamelContext(registry))
CamelContextManager.mandatoryContext.addRoutes(new CustomRouteBuilder)
CamelContextManager.start
// Internal registration
CamelContextManager.typedActorRegistry.put("tc", typedConsumer)
}
override protected def afterAll = {
CamelContextManager.stop
ActorRegistry.shutdownAll
}
feature("Communicate with an internally-registered typed actor using typed-actor-internal endpoint URIs") {
import TypedActorComponent.InternalSchema
import ExchangePattern._
scenario("two-way communication with method returning String") {
val result1 = mandatoryTemplate.requestBodyAndHeader("%s:tc?method=m2" format InternalSchema, "x", "test", "y")
val result2 = mandatoryTemplate.requestBodyAndHeader("%s:tc?method=m4" format InternalSchema, "x", "test", "y")
assert(result1 === "m2: x y")
assert(result2 === "m4: x y")
}
scenario("two-way communication with method returning void") {
val result = mandatoryTemplate.requestBodyAndHeader("%s:tc?method=m5" format InternalSchema, "x", "test", "y")
assert(result === "x") // returns initial body
}
scenario("one-way communication with method returning String") {
val result = mandatoryTemplate.send("%s:tc?method=m2" format InternalSchema, InOnly, new Processor {
def process(exchange: Exchange) = {
exchange.getIn.setBody("x")
exchange.getIn.setHeader("test", "y")
}
});
assert(result.getPattern === InOnly)
assert(result.getIn.getBody === "m2: x y")
assert(result.getOut.getBody === null)
}
scenario("one-way communication with method returning void") {
val result = mandatoryTemplate.send("%s:tc?method=m5" format InternalSchema, InOnly, new Processor {
def process(exchange: Exchange) = {
exchange.getIn.setBody("x")
exchange.getIn.setHeader("test", "y")
}
});
assert(result.getPattern === InOnly)
assert(result.getIn.getBody === "x")
assert(result.getOut.getBody === null)
}
}
feature("Communicate with an internally-registered typed actor using typed-actor endpoint URIs") {
scenario("communication not possible") {
intercept[ResolveEndpointFailedException] {
mandatoryTemplate.requestBodyAndHeader("typed-actor:tc?method=m2", "x", "test", "y")
}
}
}
feature("Communicate with an externally-registered typed actor using typed-actor endpoint URIs") {
scenario("two-way communication with method returning String") {
val result = mandatoryTemplate.requestBody("typed-actor:ta?method=foo", "test")
assert(result === "foo: test")
}
scenario("two-way communication with method returning String via custom route") {
val result = mandatoryTemplate.requestBody("direct:test", "test")
assert(result === "foo: test")
}
}
}
object TypedActorComponentFeatureTest {
class CustomRouteBuilder extends RouteBuilder {
def configure = {
from("direct:test").to("typed-actor:ta?method=foo")
}
}
}

View file

@ -1,81 +0,0 @@
package akka.camel.support
import java.util.concurrent.{TimeUnit, CountDownLatch}
import collection.mutable.Buffer
import akka.camel.Message
import akka.actor.Actor
import TestSupport._
object TestSupport {
type Handler = PartialFunction[Any, Any]
}
trait TestActor extends Actor {
def receive = {
case msg => {
handler(msg)
}
}
def handler: Handler
}
class Tester1 extends TestActor with Retain with Countdown {
def handler = retain andThen countdown
}
class Tester2 extends TestActor with Respond {
def handler = respond
}
class Tester3 extends TestActor with Noop {
self.timeout = 1
def handler = noop
}
trait Countdown { this: Actor =>
var latch: CountDownLatch = new CountDownLatch(0)
def countdown: Handler = {
case SetExpectedMessageCount(num) => {
latch = new CountDownLatch(num)
self.reply(latch)
}
case msg => latch.countDown
}
}
trait Respond { this: Actor =>
def respond: Handler = {
case msg: Message => self.reply(response(msg))
}
def response(msg: Message): Any = "Hello %s" format msg.body
}
trait Retain { this: Actor =>
val messages = Buffer[Any]()
def retain: Handler = {
case GetRetainedMessage => self.reply(messages.last)
case GetRetainedMessages(p) => self.reply(messages.toList.filter(p))
case msg => {
messages += msg
msg
}
}
}
trait Noop { this: Actor =>
def noop: Handler = {
case msg => msg
}
}
case class SetExpectedMessageCount(num: Int)
case class GetRetainedMessage()
case class GetRetainedMessages(p: Any => Boolean) {
def this() = this(_ => true)
}

View file

@ -6,16 +6,12 @@ package akka.http
import akka.config.Config
import akka.util.{Logging, Bootable}
import akka.camel.CamelService
import akka.remote.BootableRemoteActorService
import akka.actor.BootableActorLoaderService
import akka.servlet.AkkaLoader
class DefaultAkkaLoader extends AkkaLoader {
def boot(): Unit = boot(true,
new EmbeddedAppServer with BootableActorLoaderService
with BootableRemoteActorService
with CamelService)
def boot(): Unit = boot(true, new EmbeddedAppServer with BootableActorLoaderService with BootableRemoteActorService)
}

View file

@ -6,7 +6,6 @@ package akka.servlet
import akka.remote.BootableRemoteActorService
import akka.actor.BootableActorLoaderService
import akka.camel.CamelService
import akka.config.Config
import akka.util.{Logging, Bootable}
@ -30,5 +29,5 @@ class Initializer extends ServletContextListener {
loader.shutdown
def contextInitialized(e: ServletContextEvent): Unit =
loader.boot(true, new BootableActorLoaderService with BootableRemoteActorService with CamelService)
loader.boot(true, new BootableActorLoaderService with BootableRemoteActorService)
}

View file

@ -1,41 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.jta
import javax.transaction.{TransactionManager, SystemException}
import com.atomikos.icatch.jta.{J2eeTransactionManager, J2eeUserTransaction}
import com.atomikos.icatch.config.{TSInitInfo, UserTransactionService, UserTransactionServiceImp}
import akka.config.Config._
import akka.util.Duration
object AtomikosTransactionService extends AtomikosTransactionService
/**
* Atomikos implementation of the transaction service trait.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class AtomikosTransactionService extends TransactionService with TransactionProtocol {
val JTA_TRANSACTION_TIMEOUT = Duration(config.getInt("akka.jta.timeout", 60), TIME_UNIT)
private val txService: UserTransactionService = new UserTransactionServiceImp
private val info: TSInitInfo = txService.createTSInitInfo
val transactionContainer: TransactionContainer = TransactionContainer(Right(Some(
try {
txService.init(info)
val tm: TransactionManager = new J2eeTransactionManager
tm.setTransactionTimeout(JTA_TRANSACTION_TIMEOUT.toSeconds.toInt)
tm
} catch {
case e => throw new SystemException(
"Could not create a new Atomikos J2EE Transaction Manager, due to: " + e.toString)
}
)))
// TODO: gracefully postStop of the TM
//txService.postStop(false)
}

View file

@ -1,223 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.jta
import javax.transaction.{TransactionManager, UserTransaction,
Transaction => JtaTransaction, SystemException,
Status, Synchronization, TransactionSynchronizationRegistry}
import javax.naming.{InitialContext, Context, NamingException}
import akka.config.Config._
import akka.util.Logging
import akka.stm.Transaction
import akka.AkkaException
class JtaConfigurationException(message: String) extends AkkaException(message)
/**
* Detects if there is a UserTransaction or TransactionManager available in the JNDI.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object TransactionContainer extends Logging {
val AKKA_JTA_TRANSACTION_SERVICE_CLASS = "akka.jta.AtomikosTransactionService"
val DEFAULT_USER_TRANSACTION_NAME = "java:comp/UserTransaction"
val FALLBACK_TRANSACTION_MANAGER_NAMES = "java:comp/TransactionManager" ::
"java:appserver/TransactionManager" ::
"java:pm/TransactionManager" ::
"java:/TransactionManager" :: Nil
val DEFAULT_TRANSACTION_SYNCHRONIZATION_REGISTRY_NAME = "java:comp/TransactionSynchronizationRegistry"
val JTA_PROVIDER = config.getString("akka.jta.provider", "from-jndi")
private var synchronizationRegistry: Option[TransactionSynchronizationRegistry] = None
def apply(tm: Either[Option[UserTransaction], Option[TransactionManager]]) = new TransactionContainer(tm)
def apply(): TransactionContainer =
JTA_PROVIDER match {
case "from-jndi" =>
new TransactionContainer(findUserTransaction match {
case None => Right(findTransactionManager)
case tm => Left(tm)
})
case "atomikos" =>
try {
Class.forName(AKKA_JTA_TRANSACTION_SERVICE_CLASS)
.newInstance.asInstanceOf[TransactionService]
.transactionContainer
} catch {
case e: ClassNotFoundException =>
throw new JtaConfigurationException(
"JTA provider defined as 'atomikos', but the AtomikosTransactionService classes can not be found." +
"\n\tPlease make sure you have 'akka-jta' JAR and its dependencies on your classpath.")
}
case _ =>
throw new JtaConfigurationException(
"No UserTransaction on TransactionManager could be found in scope." +
"\n\tEither add 'akka-jta' to the classpath or make sure there is a" +
"\n\tTransactionManager or UserTransaction defined in the JNDI.")
}
def findUserTransaction: Option[UserTransaction] = {
val located = createInitialContext.lookup(DEFAULT_USER_TRANSACTION_NAME)
if (located eq null) None
else {
log.info("JTA UserTransaction detected [%s]", located)
Some(located.asInstanceOf[UserTransaction])
}
}
def findSynchronizationRegistry: Option[TransactionSynchronizationRegistry] = synchronized {
if (synchronizationRegistry.isDefined) synchronizationRegistry
else {
val located = createInitialContext.lookup(DEFAULT_TRANSACTION_SYNCHRONIZATION_REGISTRY_NAME)
if (located eq null) None
else {
log.info("JTA TransactionSynchronizationRegistry detected [%s]", located)
synchronizationRegistry = Some(located.asInstanceOf[TransactionSynchronizationRegistry])
synchronizationRegistry
}
}
}
def findTransactionManager: Option[TransactionManager] = {
val context = createInitialContext
val tms = for {
name <- FALLBACK_TRANSACTION_MANAGER_NAMES
tm = context.lookup(name)
if tm ne null
} yield tm
tms match {
case Nil => None
case tm :: _ =>
log.info("JTA TransactionManager detected [%s]", tm)
Some(tm.asInstanceOf[TransactionManager])
}
}
private def createInitialContext = new InitialContext(new java.util.Hashtable)
}
/**
* JTA transaction container holding either a UserTransaction or a TransactionManager.
* <p/>
* The TransactionContainer is created using the factory <tt>val container = TransactionContainer()</tt>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class TransactionContainer private (
val tm: Either[Option[UserTransaction], Option[TransactionManager]]) extends Logging {
def registerSynchronization(sync: Synchronization) = {
TransactionContainer.findSynchronizationRegistry match { // try to use SynchronizationRegistry in JNDI
case Some(registry) =>
registry.asInstanceOf[TransactionSynchronizationRegistry].registerInterposedSynchronization(sync)
case None =>
tm match {
case Right(Some(txMan)) => // try to use TransactionManager
txMan.getTransaction.registerSynchronization(sync)
case _ =>
log.warning("Cannot find TransactionSynchronizationRegistry in JNDI, can't register STM synchronization")
}
}
}
def beginWithStmSynchronization(transaction: Transaction) = {
begin
registerSynchronization(new StmSynchronization(this, transaction))
}
def begin = {
TransactionContainer.log.trace("Starting JTA transaction")
tm match {
case Left(Some(userTx)) => userTx.begin
case Right(Some(txMan)) => txMan.begin
case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
}
}
def commit = {
TransactionContainer.log.trace("Committing JTA transaction")
tm match {
case Left(Some(userTx)) => userTx.commit
case Right(Some(txMan)) => txMan.commit
case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
}
}
def rollback = {
TransactionContainer.log.trace("Aborting JTA transaction")
tm match {
case Left(Some(userTx)) => userTx.rollback
case Right(Some(txMan)) => txMan.rollback
case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
}
}
def getStatus = tm match {
case Left(Some(userTx)) => userTx.getStatus
case Right(Some(txMan)) => txMan.getStatus
case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
}
def isInExistingTransaction = tm match {
case Left(Some(userTx)) => userTx.getStatus == Status.STATUS_ACTIVE
case Right(Some(txMan)) => txMan.getStatus == Status.STATUS_ACTIVE
case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
}
def isRollbackOnly = tm match {
case Left(Some(userTx)) => userTx.getStatus == Status.STATUS_MARKED_ROLLBACK
case Right(Some(txMan)) => txMan.getStatus == Status.STATUS_MARKED_ROLLBACK
case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
}
def setRollbackOnly = tm match {
case Left(Some(userTx)) => userTx.setRollbackOnly
case Right(Some(txMan)) => txMan.setRollbackOnly
case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
}
def suspend = tm match {
case Right(Some(txMan)) => txMan.suspend
case _ => throw new JtaConfigurationException("Does not have a TransactionManager in scope")
}
def resume(tx: JtaTransaction) = tm match {
case Right(Some(txMan)) => txMan.resume(tx)
case _ => throw new JtaConfigurationException("Does not have a TransactionManager in scope")
}
}
/**
* STM Synchronization class for synchronizing with the JTA TransactionManager.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class StmSynchronization(tc: TransactionContainer, tx: Transaction) extends Synchronization with Logging {
def beforeCompletion = {
val status = tc.getStatus
if (status != Status.STATUS_ROLLEDBACK &&
status != Status.STATUS_ROLLING_BACK &&
status != Status.STATUS_MARKED_ROLLBACK) {
log.debug("JTA transaction has failed, abort STM transaction")
tx.transaction.foreach(_.abort) // abort multiverse tx
}
}
def afterCompletion(status: Int) = {}
}
/**
* JTA Transaction service.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait TransactionService {
def transactionContainer: TransactionContainer
}

View file

@ -1,238 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.jta
import javax.transaction.{Transaction, Status, TransactionManager, Synchronization}
import akka.util.Logging
import akka.config.Config._
/**
* The TransactionContext object manages the transactions.
* Can be used as higher-order functional 'atomic blocks' or monadic.
*
* Manages a thread-local stack of TransactionContexts.
* <p/>
* Example usage 1:
* <pre>
* import TransactionContext._
*
* withTxRequired {
* ... // transactional stuff
* }
* // or
* withTxRequiresNew {
* ... // transactional stuff
* }
* </pre>
* Example usage 2:
* <pre>
* for {
* ctx <- TransactionContext.Required
* entity <- updatedEntities
* if !ctx.isRollbackOnly
* } {
* // transactional stuff
* ...
* }
* </pre>
* Example usage 3:
* <pre>
* val users = for {
* ctx <- TransactionContext.Required
* name <- userNames
* } yield {
* // transactional stuff
* ...
* }
* </pre>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object TransactionContext extends TransactionProtocol with Logging {
implicit val tc = TransactionContainer()
private[TransactionContext] val stack = new scala.util.DynamicVariable(new TransactionContext(tc))
/**
* This method can be used to register a Synchronization instance for participating with the JTA transaction.
* Here is an example of how to add a JPA EntityManager integration.
* <pre>
* TransactionContext.registerSynchronization(new javax.transaction.Synchronization() {
* def beforeCompletion = {
* try {
* val status = tm.getStatus
* if (status != Status.STATUS_ROLLEDBACK &&
* status != Status.STATUS_ROLLING_BACK &&
* status != Status.STATUS_MARKED_ROLLBACK) {
* log.debug("Flushing EntityManager...")
* em.flush // flush EntityManager on success
* }
* } catch {
* case e: javax.transaction.SystemException => throw new RuntimeException(e)
* }
* }
*
* def afterCompletion(status: Int) = {
* val status = tm.getStatus
* if (closeAtTxCompletion) em.close
* if (status == Status.STATUS_ROLLEDBACK ||
* status == Status.STATUS_ROLLING_BACK ||
* status == Status.STATUS_MARKED_ROLLBACK) {
* em.close
* }
* }
* })
* </pre>
* You should also override the 'joinTransaction' and 'handleException' methods.
* See ScalaDoc for these methods in the 'TransactionProtocol' for details.
*/
def registerSynchronization(sync: Synchronization) = synchronization.add(sync)
/**
* Registeres a join transaction function.
* <p/>
* Here is an example on how to integrate with JPA EntityManager.
*
* <pre>
* TransactionContext.registerJoinTransactionFun(() => {
* val em: EntityManager = ... // get the EntityManager
* em.joinTransaction // join JTA transaction
* })
* </pre>
*/
def registerJoinTransactionFun(fn: () => Unit) = joinTransactionFuns.add(fn)
/**
* Handle exception. Can be overriden by concrete transaction service implementation.
* <p/>
* Here is an example on how to handle JPA exceptions.
*
* <pre>
* TransactionContext.registerExceptionNotToRollbackOn(classOf[NoResultException])
* TransactionContext.registerExceptionNotToRollbackOn(classOf[NonUniqueResultException])
* </pre>
*/
def registerExceptionNotToRollbackOn(e: Class[_ <: Exception]) = exceptionsNotToRollbackOn.add(e)
object Required extends TransactionMonad {
def map[T](f: TransactionMonad => T): T = withTxRequired { f(this) }
def flatMap[T](f: TransactionMonad => T): T = withTxRequired { f(this) }
def foreach(f: TransactionMonad => Unit): Unit = withTxRequired { f(this) }
}
object RequiresNew extends TransactionMonad {
def map[T](f: TransactionMonad => T): T = withTxRequiresNew { f(this) }
def flatMap[T](f: TransactionMonad => T): T = withTxRequiresNew { f(this) }
def foreach(f: TransactionMonad => Unit): Unit = withTxRequiresNew { f(this) }
}
object Supports extends TransactionMonad {
def map[T](f: TransactionMonad => T): T = withTxSupports { f(this) }
def flatMap[T](f: TransactionMonad => T): T = withTxSupports { f(this) }
def foreach(f: TransactionMonad => Unit): Unit = withTxSupports { f(this) }
}
object Mandatory extends TransactionMonad {
def map[T](f: TransactionMonad => T): T = withTxMandatory { f(this) }
def flatMap[T](f: TransactionMonad => T): T = withTxMandatory { f(this) }
def foreach(f: TransactionMonad => Unit): Unit = withTxMandatory { f(this) }
}
object Never extends TransactionMonad {
def map[T](f: TransactionMonad => T): T = withTxNever { f(this) }
def flatMap[T](f: TransactionMonad => T): T = withTxNever { f(this) }
def foreach(f: TransactionMonad => Unit): Unit = withTxNever { f(this) }
}
object NoOpTransactionMonad extends TransactionMonad {
def map[T](f: TransactionMonad => T): T = f(this)
def flatMap[T](f: TransactionMonad => T): T = f(this)
def foreach(f: TransactionMonad => Unit): Unit = f(this)
override def filter(f: TransactionMonad => Boolean): TransactionMonad = this
}
private[jta] def setRollbackOnly = current.setRollbackOnly
private[jta] def isRollbackOnly = current.isRollbackOnly
private[jta] def getTransactionContainer: TransactionContainer = current.getTransactionContainer
private[this] def current = stack.value
/**
* Continues with the invocation defined in 'body' with the brand new context define in 'newCtx', the old
* one is put on the stack and will automatically come back in scope when the method exits.
* <p/>
* Suspends and resumes the current JTA transaction.
*/
private[jta] def withNewContext[T](body: => T): T = {
val suspendedTx: Option[Transaction] =
if (getTransactionContainer.isInExistingTransaction) {
log.debug("Suspending TX")
Some(getTransactionContainer.suspend)
} else None
val result = stack.withValue(new TransactionContext(tc)) { body }
if (suspendedTx.isDefined) {
log.debug("Resuming TX")
getTransactionContainer.resume(suspendedTx.get)
}
result
}
}
/**
* Base monad for the transaction monad implementations.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait TransactionMonad {
// -----------------------------
// Monadic definitions
// -----------------------------
def map[T](f: TransactionMonad => T): T
def flatMap[T](f: TransactionMonad => T): T
def foreach(f: TransactionMonad => Unit): Unit
def filter(f: TransactionMonad => Boolean): TransactionMonad =
if (f(this)) this else TransactionContext.NoOpTransactionMonad
// -----------------------------
// JTA Transaction definitions
// -----------------------------
/**
* Marks the current transaction as doomed.
*/
def setRollbackOnly = TransactionContext.setRollbackOnly
/**
* Marks the current transaction as doomed.
*/
def doom = TransactionContext.setRollbackOnly
/**
* Checks if the current transaction is doomed.
*/
def isRollbackOnly = TransactionContext.isRollbackOnly
/**
* Checks that the current transaction is NOT doomed.
*/
def isNotDoomed = !TransactionContext.isRollbackOnly
}
/**
* Transaction context, holds the EntityManager and the TransactionManager.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class TransactionContext(val tc: TransactionContainer) {
def registerSynchronization(sync: Synchronization) = TransactionContext.registerSynchronization(sync)
def setRollbackOnly = tc.setRollbackOnly
def isRollbackOnly: Boolean = tc.getStatus == Status.STATUS_MARKED_ROLLBACK
def getTransactionContainer: TransactionContainer = tc
}

View file

@ -1,227 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.jta
import akka.util.Logging
import java.util.{List => JList}
import java.util.concurrent.CopyOnWriteArrayList
import javax.naming.{NamingException, Context, InitialContext}
import javax.transaction.{
Transaction,
UserTransaction,
TransactionManager,
Status,
RollbackException,
SystemException,
Synchronization,
TransactionRequiredException
}
/**
* <p>
* Trait that implements a JTA transaction service that obeys the transaction semantics defined
* in the transaction attribute types for the transacted methods according to the EJB 3 draft specification.
* The aspect handles UserTransaction, TransactionManager instance variable injection thru @javax.ejb.Inject
* (name subject to change as per EJB 3 spec) and method transaction levels thru @javax.ejb.TransactionAttribute.
* </p>
*
* <p>
* This trait should be inherited to implement the getTransactionManager() method that should return a concrete
* javax.transaction.TransactionManager implementation (from JNDI lookup etc).
* </p>
* <p>
* <h3>Transaction attribute semantics</h3>
* (From http://www.kevinboone.com/ejb-transactions.html)
* </p>
* <p>
* <h4>Required</h4>
* 'Required' is probably the best choice (at least initially) for an EJB method that will need to be transactional. In this case, if the method's caller is already part of a transaction, then the EJB method does not create a new transaction, but continues in the same transaction as its caller. If the caller is not in a transaction, then a new transaction is created for the EJB method. If something happens in the EJB that means that a rollback is required, then the extent of the rollback will include everything done in the EJB method, whatever the condition of the caller. If the caller was in a transaction, then everything done by the caller will be rolled back as well. Thus the 'required' attribute ensures that any work done by the EJB will be rolled back if necessary, and if the caller requires a rollback that too will be rolled back.
* </p>
* <p>
* <h4>RequiresNew</h4>
* 'RequiresNew' will be appropriate if you want to ensure that the EJB method is rolled back if necessary, but you don't want the rollback to propogate back to the caller. This attribute results in the creation of a new transaction for the method, regardless of the transactional state of the caller. If the caller was operating in a transaction, then its transaction is suspended until the EJB method completes. Because a new transaction is always created, there may be a slight performance penalty if this attribute is over-used.
* </p>
* <p>
* <h4>Mandatory</h4>
* With the 'mandatory' attribute, the EJB method will not even start unless its caller is in a transaction. It will throw a <code>TransactionRequiredException</code> instead. If the method does start, then it will become part of the transaction of the caller. So if the EJB method signals a failure, the caller will be rolled back as well as the EJB.
* </p>
* <p>
* <h4>Supports</h4>
* With this attribute, the EJB method does not care about the transactional context of its caller. If the caller is part of a transaction, then the EJB method will be part of the same transaction. If the EJB method fails, the transaction will roll back. If the caller is not part of a transaction, then the EJB method will still operate, but a failure will not cause anything to roll back. 'Supports' is probably the attribute that leads to the fastest method call (as there is no transactional overhead), but it can lead to unpredicatable results. If you want a method to be isolated from transactions, that is, to have no effect on the transaction of its caller, then use 'NotSupported' instead.
* </p>
* <p>
* <h4>NotSupported</h4>
* With the 'NotSupported' attribute, the EJB method will never take part in a transaction. If the caller is part of a transaction, then the caller's transaction is suspended. If the EJB method fails, there will be no effect on the caller's transaction, and no rollback will occur. Use this method if you want to ensure that the EJB method will not cause a rollback in its caller. This is appropriate if, for example, the method does something non-essential, such as logging a message. It would not be helpful if the failure of this operation caused a transaction rollback.
* </p>
* <p>
* <h4>Never</h4>
* The 'NotSupported'' attribute will ensure that the EJB method is never called by a transactional caller. Any attempt to do so will result in a <code>RemoteException</code> being thrown. This attribute is probably less useful than `NotSupported', in that NotSupported will assure that the caller's transaction is never affected by the EJB method (just as `Never' does), but will allow a call from a transactional caller if necessary.
* </p>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait TransactionProtocol extends Logging {
protected val synchronization: JList[Synchronization] = new CopyOnWriteArrayList[Synchronization]
protected val joinTransactionFuns: JList[() => Unit] = new CopyOnWriteArrayList[() => Unit]
protected val exceptionsNotToRollbackOn: JList[Class[_ <: Exception]] = new CopyOnWriteArrayList[Class[_ <: Exception]]
def joinTransaction(): Unit = {
val it = joinTransactionFuns.iterator
while (it.hasNext) {
val fn = it.next
fn()
}
}
def handleException(tm: TransactionContainer, e: Exception) = {
var rollback = true
val it = joinTransactionFuns.iterator
while (it.hasNext) {
val exception = it.next
if (e.getClass.isAssignableFrom(exception.getClass))
rollback = false
}
if (rollback) tm.setRollbackOnly
throw e
}
/**
* Wraps body in a transaction with REQUIRED semantics.
* <p/>
* Creates a new transaction if no transaction is active in scope, else joins the outer transaction.
*/
def withTxRequired[T](body: => T): T = {
val tm = TransactionContext.getTransactionContainer
if (!isInExistingTransaction(tm)) {
tm.begin
registerSynchronization
try {
joinTransaction
body
} catch {
case e: Exception => handleException(tm, e)
} finally {
commitOrRollBack(tm)
}
} else body
}
/**
* Wraps body in a transaction with REQUIRES_NEW semantics.
* <p/>
* Suspends existing transaction, starts a new transaction, invokes body,
* commits or rollbacks new transaction, finally resumes previous transaction.
*/
def withTxRequiresNew[T](body: => T): T = TransactionContext.withNewContext {
val tm = TransactionContext.getTransactionContainer
tm.begin
registerSynchronization
try {
joinTransaction
body
} catch {
case e: Exception => handleException(tm, e)
} finally {
commitOrRollBack(tm)
}
}
/**
* Wraps body in a transaction with NOT_SUPPORTED semantics.
* <p/>
* Suspends existing transaction, invokes body, resumes transaction.
*/
def withTxNotSupported[T](body: => T): T = TransactionContext.withNewContext {
body
}
/**
* Wraps body in a transaction with SUPPORTS semantics.
* <p/>
* Basicalla a No-op.
*/
def withTxSupports[T](body: => T): T = {
// attach to current if exists else skip -> do nothing
body
}
/**
* Wraps body in a transaction with MANDATORY semantics.
* <p/>
* Throws a TransactionRequiredException if there is no transaction active in scope.
*/
def withTxMandatory[T](body: => T): T = {
if (!isInExistingTransaction(TransactionContext.getTransactionContainer))
throw new TransactionRequiredException("No active TX at method with TX type set to MANDATORY")
body
}
/**
* Wraps body in a transaction with NEVER semantics.
* <p/>
* Throws a SystemException in case of an existing transaction in scope.
*/
def withTxNever[T](body: => T): T = {
if (isInExistingTransaction(TransactionContext.getTransactionContainer))
throw new SystemException("Detected active TX at method with TX type set to NEVER")
body
}
protected def commitOrRollBack(tm: TransactionContainer) = {
if (isInExistingTransaction(tm)) {
if (isRollbackOnly(tm)) {
log.debug("Rolling back TX marked as ROLLBACK_ONLY")
tm.rollback
} else {
log.debug("Committing TX")
tm.commit
}
}
}
// ---------------------------
// Helper methods
// ---------------------------
protected def registerSynchronization = {
val it = synchronization.iterator
while (it.hasNext) TransactionContext.getTransactionContainer.registerSynchronization(it.next)
}
/**
* Checks if a transaction is an existing transaction.
*
* @param tm the transaction manager
* @return boolean
*/
protected def isInExistingTransaction(tm: TransactionContainer): Boolean =
tm.getStatus != Status.STATUS_NO_TRANSACTION
/**
* Checks if current transaction is set to rollback only.
*
* @param tm the transaction manager
* @return boolean
*/
protected def isRollbackOnly(tm: TransactionContainer): Boolean =
tm.getStatus == Status.STATUS_MARKED_ROLLBACK
/**
* A ThreadLocal variable where to store suspended TX and enable pay as you go
* before advice - after advice data sharing in a specific case of requiresNew TX
*/
private val suspendedTx = new ThreadLocal[Transaction] {
override def initialValue = null
}
private def storeInThreadLocal(tx: Transaction) = suspendedTx.set(tx)
private def fetchFromThreadLocal: Option[Transaction] = {
if ((suspendedTx ne null) && (suspendedTx.get() ne null)) Some(suspendedTx.get.asInstanceOf[Transaction])
else None
}
}

View file

@ -1,16 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.jta
import org.scalatest.junit.JUnitSuite
import org.junit.Test
import akka.stm.ReflectiveJtaModule
class ReflectiveAccessSpec extends JUnitSuite {
@Test def ensureReflectiveAccessCanLoadTransactionContainer {
ReflectiveJtaModule.ensureJtaEnabled
assert(ReflectiveJtaModule.transactionContainerObjectInstance.isDefined)
}
}

View file

@ -1,22 +0,0 @@
<?xml version="1.0" encoding="UTF-8"?>
<features name="akka 0.9">
<feature name="scala-osgi" version="1.5">
<bundle>mvn:com.weiglewilczek.scala-lang-osgi/scala-library/2.8.0.RC2</bundle>
<bundle>mvn:org.eclipse.scalamodules/scalamodules-core/2.0-M2</bundle>
</feature>
<feature name="sjson" version="0.6-SNAPSHOT">
<bundle>mvn:akka.akka-wrap/dispatch-json_2.8.0.RC3_osgi/0.7.4</bundle>
<bundle>mvn:org.objenesis/objenesis/1.2</bundle>
<bundle>mvn:sjson.json/sjson/0.6-SNAPSHOT</bundle>
</feature>
<feature name="akka-remote" version="0.9-SNAPSHOT">
<feature>sjson</feature>
<bundle>mvn:akka.akka-wrap/jgroups-wrapper_2.8.0.RC3_osgi/2.9.0.GA</bundle>
<bundle>mvn:org.jboss.netty/netty/3.2.0.CR1</bundle>
<bundle>mvn:akka/akka-remote_2.8.0.RC3_osgi/0.9</bundle>
</feature>
</features>

View file

@ -1,25 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.kernel
import akka.http.{ EmbeddedAppServer, DefaultAkkaLoader }
import akka.remote.BootableRemoteActorService
object Main {
def main(args: Array[String]) = Kernel.boot
}
/**
* The Akka Kernel, is used to start And postStop Akka in standalone/kernel mode.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Kernel extends DefaultAkkaLoader {
//For testing purposes only
def startRemoteService(): Unit = bundles.foreach( _ match {
case x: BootableRemoteActorService => x.startRemoteService
case _ =>
})
}

View file

@ -1,199 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.persistence.cassandra
import java.io.{Flushable, Closeable}
import akka.persistence.common._
import akka.util.Logging
import akka.util.Helpers._
import akka.serialization.Serializer
import akka.config.Config.config
import scala.collection.mutable.Map
import org.apache.cassandra.db.ColumnFamily
import org.apache.cassandra.thrift._
import org.apache.thrift.transport._
import org.apache.thrift.protocol._
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait CassandraSession extends Closeable with Flushable {
import scala.collection.JavaConversions._
import java.util.{Map => JMap, List => JList}
val client: Cassandra.Client
val keyspace: String
val obtainedAt: Long
val consistencyLevel: ConsistencyLevel
val schema: JMap[String, JMap[String, String]]
/**
* Count is always the max number of results to return.
So it means, starting with `start`, or the first one if start is
empty, go until you hit `finish` or `count`, whichever comes first.
Empty is not a legal column name so if finish is empty it is ignored
and only count is used.
We don't offer a numeric offset since that can't be supported
efficiently with a log-structured merge disk format.
*/
// ====================================
// ====== Scala-style API names
// ====================================
def /(key: String, columnParent: ColumnParent, start: Array[Byte], end: Array[Byte], ascending: Boolean, count: Int): List[ColumnOrSuperColumn] =
/(key, columnParent, start, end, ascending, count, consistencyLevel)
def /(key: String, columnParent: ColumnParent, start: Array[Byte], end: Array[Byte], ascending: Boolean, count: Int, consistencyLevel: ConsistencyLevel): List[ColumnOrSuperColumn] = {
val slicePredicate = new SlicePredicate
slicePredicate.setSlice_range(new SliceRange(start, end, ascending, count))
client.get_slice(keyspace, key, columnParent, slicePredicate, consistencyLevel).toList
}
def /(key: String, columnParent: ColumnParent, slicePredicate: SlicePredicate): List[ColumnOrSuperColumn] =
client.get_slice(keyspace, key, columnParent, slicePredicate, consistencyLevel).toList
def /(key: String, columnParent: ColumnParent, slicePredicate: SlicePredicate, consistencyLevel: ConsistencyLevel): List[ColumnOrSuperColumn] =
client.get_slice(keyspace, key, columnParent, slicePredicate, consistencyLevel).toList
def |(key: String, colPath: ColumnPath): Option[ColumnOrSuperColumn] =
|(key, colPath, consistencyLevel)
def |(key: String, colPath: ColumnPath, consistencyLevel: ConsistencyLevel): Option[ColumnOrSuperColumn] =
client.get(keyspace, key, colPath, consistencyLevel)
def |#(key: String, columnParent: ColumnParent): Int =
|#(key, columnParent, consistencyLevel)
def |#(key: String, columnParent: ColumnParent, consistencyLevel: ConsistencyLevel): Int =
client.get_count(keyspace, key, columnParent, consistencyLevel)
def ++|(key: String, colPath: ColumnPath, value: Array[Byte]): Unit =
++|(key, colPath, value, obtainedAt, consistencyLevel)
def ++|(key: String, colPath: ColumnPath, value: Array[Byte], consistencyLevel: ConsistencyLevel): Unit =
++|(key, colPath, value, obtainedAt, consistencyLevel)
def ++|(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long): Unit =
++|(key, colPath, value, timestamp, consistencyLevel)
def ++|(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long, consistencyLevel: ConsistencyLevel) =
client.insert(keyspace, key, colPath, value, timestamp, consistencyLevel)
def ++|(key: String, batch: Map[String, List[ColumnOrSuperColumn]]): Unit =
++|(key, batch, consistencyLevel)
def ++|(key: String, batch: Map[String, List[ColumnOrSuperColumn]], consistencyLevel: ConsistencyLevel): Unit = {
val jmap = new java.util.HashMap[String, JList[ColumnOrSuperColumn]]
for (entry <- batch; (key, value) = entry) jmap.put(key, new java.util.ArrayList(value))
client.batch_insert(keyspace, key, jmap, consistencyLevel)
}
def --(key: String, columnPath: ColumnPath, timestamp: Long): Unit =
--(key, columnPath, timestamp, consistencyLevel)
def --(key: String, columnPath: ColumnPath, timestamp: Long, consistencyLevel: ConsistencyLevel): Unit =
client.remove(keyspace, key, columnPath, timestamp, consistencyLevel)
// ====================================
// ====== Java-style API names
// ====================================
def getSlice(key: String, columnParent: ColumnParent, start: Array[Byte], end: Array[Byte], ascending: Boolean, count: Int) = / (key, columnParent, start, end, ascending, count, consistencyLevel)
def getSlice(key: String, columnParent: ColumnParent, start: Array[Byte], end: Array[Byte], ascending: Boolean, count: Int, consistencyLevel: ConsistencyLevel) = / (key, columnParent, start, end, ascending, count, consistencyLevel)
def getSlice(key: String, columnParent: ColumnParent, slicePredicate: SlicePredicate) = / (key, columnParent, slicePredicate)
def getSlice(key: String, columnParent: ColumnParent, slicePredicate: SlicePredicate, consistencyLevel: ConsistencyLevel) = / (key, columnParent, slicePredicate, consistencyLevel)
def get(key: String, colPath: ColumnPath) = |(key, colPath)
def get(key: String, colPath: ColumnPath, consistencyLevel: ConsistencyLevel) = |(key, colPath, consistencyLevel)
def getCount(key: String, columnParent: ColumnParent)= |#(key, columnParent)
def getCount(key: String, columnParent: ColumnParent, consistencyLevel: ConsistencyLevel) = |#(key, columnParent, consistencyLevel)
def insert(key: String, colPath: ColumnPath, value: Array[Byte]): Unit = ++|(key, colPath, value)
def insert(key: String, colPath: ColumnPath, value: Array[Byte], consistencyLevel: ConsistencyLevel): Unit = ++|(key, colPath, value, consistencyLevel)
def insert(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long): Unit = ++|(key, colPath, value, timestamp)
def insert(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long, consistencyLevel: ConsistencyLevel) = ++|(key, colPath, value, timestamp, consistencyLevel)
def insert(key: String, batch: Map[String, List[ColumnOrSuperColumn]]): Unit = ++|(key, batch)
def insert(key: String, batch: Map[String, List[ColumnOrSuperColumn]], consistencyLevel: ConsistencyLevel): Unit = ++|(key, batch, consistencyLevel)
def remove(key: String, columnPath: ColumnPath, timestamp: Long): Unit = --(key, columnPath, timestamp)
def remove(key: String, columnPath: ColumnPath, timestamp: Long, consistencyLevel: ConsistencyLevel): Unit = --(key, columnPath, timestamp, consistencyLevel)
}
class CassandraSessionPool[T <: TTransport](
space: String,
transportPool: Pool[T],
inputProtocol: Protocol,
outputProtocol: Protocol,
consistency: ConsistencyLevel) extends Closeable with Logging {
def this(space: String, transportPool: Pool[T], ioProtocol: Protocol, consistency: ConsistencyLevel) =
this (space, transportPool, ioProtocol, ioProtocol, consistency)
def newSession: CassandraSession = newSession(consistency)
def newSession(consistencyLevel: ConsistencyLevel): CassandraSession = {
val socket = transportPool.borrowObject
val cassandraClient = new Cassandra.Client(inputProtocol(socket), outputProtocol(socket))
val cassandraSchema = cassandraClient.describe_keyspace(space)
new CassandraSession {
val keyspace = space
val client = cassandraClient
val obtainedAt = System.currentTimeMillis
val consistencyLevel = consistency
val schema = cassandraSchema
log.debug("Creating %s", toString)
def flush = socket.flush
def close = transportPool.returnObject(socket)
override def toString = "[CassandraSession]\n\tkeyspace = " + keyspace + "\n\tschema = " + schema
}
}
def withSession[T](body: CassandraSession => T) = {
val session = newSession(consistency)
try {
val result = body(session)
session.flush
result
} finally {
session.close
}
}
def close = transportPool.close
}
sealed abstract class Protocol(val factory: TProtocolFactory) {
def apply(transport: TTransport) = factory.getProtocol(transport)
}
object Protocol {
object Binary extends Protocol(new TBinaryProtocol.Factory)
object SimpleJSON extends Protocol(new TSimpleJSONProtocol.Factory)
object JSON extends Protocol(new TJSONProtocol.Factory)
}

View file

@ -1,59 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.persistence.cassandra
import akka.stm._
import akka.persistence.common._
import akka.actor.{newUuid}
object CassandraStorage extends Storage {
type ElementType = Array[Byte]
def newMap: PersistentMap[ElementType, ElementType] = newMap(newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(newUuid.toString)
override def newQueue: PersistentQueue[ElementType] = newQueue(newUuid.toString)
def getMap(id: String): PersistentMap[ElementType, ElementType] = newMap(id)
def getVector(id: String): PersistentVector[ElementType] = newVector(id)
def getRef(id: String): PersistentRef[ElementType] = newRef(id)
override def getQueue(id: String): PersistentQueue[ElementType] = newQueue(id)
def newMap(id: String): PersistentMap[ElementType, ElementType] = new CassandraPersistentMap(id)
def newVector(id: String): PersistentVector[ElementType] = new CassandraPersistentVector(id)
def newRef(id: String): PersistentRef[ElementType] = new CassandraPersistentRef(id)
override def newQueue(id: String): PersistentQueue[ElementType] = new CassandraPersistentQueue(id)
}
/**
* Implements a persistent transactional map based on the Cassandra distributed P2P key-value storage.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class CassandraPersistentMap(id: String) extends PersistentMapBinary {
val uuid = id
val storage = CassandraStorageBackend
}
/**
* Implements a persistent transactional vector based on the Cassandra
* distributed P2P key-value storage.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class CassandraPersistentVector(id: String) extends PersistentVector[Array[Byte]] {
val uuid = id
val storage = CassandraStorageBackend
}
class CassandraPersistentRef(id: String) extends PersistentRef[Array[Byte]] {
val uuid = id
val storage = CassandraStorageBackend
}
class CassandraPersistentQueue(id: String) extends PersistentQueue[Array[Byte]] {
val uuid = id
val storage = CassandraStorageBackend
}

View file

@ -1,161 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.persistence.cassandra
import akka.stm._
import akka.persistence.common._
import akka.util.Logging
import akka.util.Helpers._
import akka.config.Config.config
import org.apache.cassandra.thrift._
import java.lang.String
import collection.JavaConversions
import collection.immutable.{TreeMap, Iterable}
import java.util.{Map => JMap, HashMap => JHMap, List => JList, ArrayList => JAList}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
private[akka] object CassandraStorageBackend extends CommonStorageBackend {
import CommonStorageBackend._
type ElementType = Array[Byte]
val KEYSPACE = "akka"
val MAP_COLUMN_PARENT = new ColumnParent("map")
val VECTOR_COLUMN_PARENT = new ColumnParent("vector")
val REF_COLUMN_PARENT = new ColumnParent("ref")
val QUEUE_COLUMN_PARENT = new ColumnParent("queue")
val REF_KEY = "item".getBytes("UTF-8")
val EMPTY_BYTE_ARRAY = new Array[Byte](0)
val CASSANDRA_SERVER_HOSTNAME = config.getString("akka.persistence.cassandra.hostname", "127.0.0.1")
val CASSANDRA_SERVER_PORT = config.getInt("akka.persistence.cassandra.port", 9160)
val CONSISTENCY_LEVEL = {
config.getString("akka.persistence.cassandra.consistency-level", "QUORUM") match {
case "ZERO" => ConsistencyLevel.ZERO
case "ONE" => ConsistencyLevel.ONE
case "QUORUM" => ConsistencyLevel.QUORUM
case "DCQUORUM" => ConsistencyLevel.DCQUORUM
case "DCQUORUMSYNC" => ConsistencyLevel.DCQUORUMSYNC
case "ALL" => ConsistencyLevel.ALL
case "ANY" => ConsistencyLevel.ANY
case unknown => throw new IllegalArgumentException(
"Cassandra consistency level [" + unknown + "] is not supported." +
"\n\tExpected one of [ZERO, ONE, QUORUM, DCQUORUM, DCQUORUMSYNC, ALL, ANY] in the akka.conf configuration file.")
}
}
val IS_ASCENDING = true
@volatile private[this] var isRunning = false
private[this] val protocol: Protocol = Protocol.Binary
private[this] val sessions = new CassandraSessionPool(
KEYSPACE,
StackPool(SocketProvider(CASSANDRA_SERVER_HOSTNAME, CASSANDRA_SERVER_PORT)),
protocol,
CONSISTENCY_LEVEL)
class CassandraAccess(parent: ColumnParent) extends CommonStorageBackendAccess {
def path(key: Array[Byte]): ColumnPath = {
new ColumnPath(parent.getColumn_family).setColumn(key)
}
def delete(owner: String, key: Array[Byte]) = {
sessions.withSession{
session => {
session -- (owner, path(key), System.currentTimeMillis, CONSISTENCY_LEVEL)
}
}
}
override def getAll(owner: String, keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] = {
sessions.withSession{
session => {
var predicate = new SlicePredicate().setColumn_names(JavaConversions.asJavaList(keys.toList))
val cols = session / (owner, parent, predicate, CONSISTENCY_LEVEL)
var map = new TreeMap[Array[Byte], Array[Byte]]()(ordering)
cols.foreach{
cosc => map += cosc.getColumn.getName -> cosc.getColumn.getValue
}
map
}
}
}
def get(owner: String, key: Array[Byte], default: Array[Byte]) = {
sessions.withSession{
session => {
try
{
session | (owner, path(key), CONSISTENCY_LEVEL) match {
case Some(cosc) => cosc.getColumn.getValue
case None => default
}
} catch {
case e: NotFoundException => default
}
}
}
}
def put(owner: String, key: Array[Byte], value: Array[Byte]) = {
sessions.withSession{
session => {
session ++| (owner, path(key), value, System.currentTimeMillis, CONSISTENCY_LEVEL)
}
}
}
def drop() = {
sessions.withSession{
session => {
val slices = session.client.get_range_slices(session.keyspace, parent,
new SlicePredicate().setSlice_range(new SliceRange().setStart(Array.empty[Byte]).setFinish(Array.empty[Byte])),
new KeyRange().setStart_key("").setEnd_key(""), CONSISTENCY_LEVEL)
val mutations = new JHMap[String, JMap[String, JList[Mutation]]]
JavaConversions.asScalaIterable(slices).foreach{
keySlice: KeySlice => {
val key = keySlice.getKey
val keyMutations = JavaConversions.asScalaMap(mutations).getOrElse(key, {
val km = new JHMap[String, JList[Mutation]]
mutations.put(key, km)
km
})
val amutation = new JAList[Mutation]
val cols = new JAList[Array[Byte]]
keyMutations.put(parent.getColumn_family, amutation)
JavaConversions.asScalaIterable(keySlice.getColumns) foreach {
cosc: ColumnOrSuperColumn => {
cols.add(cosc.getColumn.getName)
}
}
amutation.add(new Mutation().setDeletion(new Deletion(System.currentTimeMillis).setPredicate(new SlicePredicate().setColumn_names(cols))))
}
}
session.client.batch_mutate(session.keyspace, mutations, CONSISTENCY_LEVEL)
}
}
}
}
def queueAccess = new CassandraAccess(QUEUE_COLUMN_PARENT)
def mapAccess = new CassandraAccess(MAP_COLUMN_PARENT)
def vectorAccess = new CassandraAccess(VECTOR_COLUMN_PARENT)
def refAccess = new CassandraAccess(REF_COLUMN_PARENT)
}

View file

@ -1,25 +0,0 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
log4j.rootLogger=DEBUG,R
# rolling log file ("system.log
log4j.appender.R=org.apache.log4j.DailyRollingFileAppender
log4j.appender.R.DatePattern='.'yyyy-MM-dd-HH
log4j.appender.R.layout=org.apache.log4j.PatternLayout
log4j.appender.R.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
log4j.appender.R.File=target/logs/system.log

View file

@ -1,337 +0,0 @@
<!--
~ Licensed to the Apache Software Foundation (ASF) under one
~ or more contributor license agreements. See the NOTICE file
~ distributed with this work for additional information
~ regarding copyright ownership. The ASF licenses this file
~ to you under the Apache License, Version 2.0 (the
~ "License"); you may not use this file except in compliance
~ with the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing,
~ software distributed under the License is distributed on an
~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
~ KIND, either express or implied. See the License for the
~ specific language governing permissions and limitations
~ under the License.
-->
<Storage>
<!--======================================================================-->
<!-- Basic Configuration -->
<!--======================================================================-->
<!--
~ The name of this cluster. This is mainly used to prevent machines in
~ one logical cluster from joining another.
-->
<ClusterName>akka</ClusterName>
<!--
~ Turn on to make new [non-seed] nodes automatically migrate the right data
~ to themselves. (If no InitialToken is specified, they will pick one
~ such that they will get half the range of the most-loaded node.)
~ If a node starts up without bootstrapping, it will mark itself bootstrapped
~ so that you can't subsequently accidently bootstrap a node with
~ data on it. (You can reset this by wiping your data and commitlog
~ directories.)
~
~ Off by default so that new clusters and upgraders from 0.4 don't
~ bootstrap immediately. You should turn this on when you start adding
~ new nodes to a cluster that already has data on it. (If you are upgrading
~ from 0.4, start your cluster with it off once before changing it to true.
~ Otherwise, no data will be lost but you will incur a lot of unnecessary
~ I/O before your cluster starts up.)
-->
<AutoBootstrap>false</AutoBootstrap>
<!--
~ Keyspaces and ColumnFamilies:
~ A ColumnFamily is the Cassandra concept closest to a relational
~ table. Keyspaces are separate groups of ColumnFamilies. Except in
~ very unusual circumstances you will have one Keyspace per application.
~ There is an implicit keyspace named 'system' for Cassandra internals.
-->
<Keyspaces>
<Keyspace Name="akka">
<!-- The fraction of keys per sstable whose locations we
keep in memory in "mostly LRU" order. (JUST the key
locations, NOT any column values.)
The amount of memory used by the default setting of
0.01 is comparable to the amount used by the internal
per-sstable key index. Consider increasing this is
fine if you have fewer, wider rows. Set to 0 to
disable entirely.
-->
<KeysCachedFraction>0.01</KeysCachedFraction>
<!--
The CompareWith attribute tells Cassandra how to sort the columns
for slicing operations. For backwards compatibility, the default
is to use AsciiType, which is probably NOT what you want.
Other options are BytesType, UTF8Type, LexicalUUIDType, TimeUUIDType, and LongType.
You can also specify the fully-qualified class name to a class
of your choice implementing org.apache.cassandra.db.marshal.IType.
SuperColumns have a similar CompareSubcolumnsWith attribute.
ByteType: simple sort by byte value. No validation is performed.
AsciiType: like BytesType, but validates that the input can be parsed as US-ASCII.
UTF8Type: A string encoded as UTF8
LongType: A 64bit long
LexicalUUIDType: a 128bit UUID, compared lexically (by byte value)
TimeUUIDType: a 128bit version 1 UUID, compared by timestamp
(To get the closest approximation to 0.3-style supercolumns,
you would use CompareWith=UTF8Type CompareSubcolumnsWith=LongType.)
if FlushPeriodInMinutes is configured and positive, it will be
flushed to disk with that period whether it is dirty or not.
This is intended for lightly-used columnfamilies so that they
do not prevent commitlog segments from being purged.
-->
<ColumnFamily CompareWith="UTF8Type" Name="map"/>
<!-- FIXME: change vector to a super column -->
<ColumnFamily CompareWith="UTF8Type" Name="vector"/>
<ColumnFamily CompareWith="UTF8Type" Name="ref"/>
<!--ColumnFamily CompareWith="UTF8Type" Name="Standard1" FlushPeriodInMinutes="60"/>
<ColumnFamily CompareWith="TimeUUIDType" Name="StandardByUUID1"/>
<ColumnFamily ColumnType="Super" CompareWith="UTF8Type" CompareSubcolumnsWith="UTF8Type" Name="Super1"/-->
</Keyspace>
</Keyspaces>
<!--
~ Partitioner: any IPartitioner may be used, including your own as long
~ as it is on the classpath. Out of the box, Cassandra provides
~ org.apache.cassandra.dht.RandomPartitioner,
~ org.apache.cassandra.dht.OrderPreservingPartitioner, and
~ org.apache.cassandra.dht.CollatingOrderPreservingPartitioner.
~ (CollatingOPP colates according to EN,US rules, not naive byte
~ ordering. Use this as an example if you need locale-aware collation.)
~ Range queries require using an order-preserving partitioner.
~
~ Achtung! Changing this parameter requires wiping your data
~ directories, since the partitioner can modify the sstable on-disk
~ format.
-->
<Partitioner>org.apache.cassandra.dht.RandomPartitioner</Partitioner>
<!--
~ If you are using an order-preserving partitioner and you know your key
~ distribution, you can specify the token for this node to use. (Keys
~ are sent to the node with the "closest" token, so distributing your
~ tokens equally along the key distribution space will spread keys
~ evenly across your cluster.) This setting is only checked the first
~ time a node is started.
~ This can also be useful with RandomPartitioner to force equal spacing
~ of tokens around the hash space, especially for clusters with a small
~ number of nodes.
-->
<InitialToken></InitialToken>
<!--
~ EndPointSnitch: Setting this to the class that implements
~ IEndPointSnitch which will see if two endpoints are in the same data
~ center or on the same rack. Out of the box, Cassandra provides
~ org.apache.cassandra.locator.EndPointSnitch
-->
<EndPointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndPointSnitch>
<!--
~ Strategy: Setting this to the class that implements
~ IReplicaPlacementStrategy will change the way the node picker works.
~ Out of the box, Cassandra provides
~ org.apache.cassandra.locator.RackUnawareStrategy and
~ org.apache.cassandra.locator.RackAwareStrategy (place one replica in
~ a different datacenter, and the others on different racks in the same
~ one.)
-->
<ReplicaPlacementStrategy>org.apache.cassandra.locator.RackUnawareStrategy</ReplicaPlacementStrategy>
<!-- Number of replicas of the data -->
<ReplicationFactor>1</ReplicationFactor>
<!--
~ Directories: Specify where Cassandra should store different data on
~ disk. Keep the data disks and the CommitLog disks separate for best
~ performance
-->
<CommitLogDirectory>target/cassandra/commitlog</CommitLogDirectory>
<DataFileDirectories>
<DataFileDirectory>target/cassandra/data</DataFileDirectory>
</DataFileDirectories>
<CalloutLocation>target/cassandra/callouts</CalloutLocation>
<StagingFileDirectory>target/cassandra/staging</StagingFileDirectory>
<!--
~ Addresses of hosts that are deemed contact points. Cassandra nodes
~ use this list of hosts to find each other and learn the topology of
~ the ring. You must change this if you are running multiple nodes!
-->
<Seeds>
<Seed>127.0.0.1</Seed>
</Seeds>
<!-- Miscellaneous -->
<!-- Time to wait for a reply from other nodes before failing the command -->
<RpcTimeoutInMillis>5000</RpcTimeoutInMillis>
<!-- Size to allow commitlog to grow to before creating a new segment -->
<CommitLogRotationThresholdInMB>128</CommitLogRotationThresholdInMB>
<!-- Local hosts and ports -->
<!--
~ Address to bind to and tell other nodes to connect to. You _must_
~ change this if you want multiple nodes to be able to communicate!
~
~ Leaving it blank leaves it up to InetAddress.getLocalHost(). This
~ will always do the Right Thing *if* the node is properly configured
~ (hostname, name resolution, etc), and the Right Thing is to use the
~ address associated with the hostname (it might not be).
-->
<ListenAddress>localhost</ListenAddress>
<!-- TCP port, for commands and data -->
<StoragePort>7000</StoragePort>
<!-- UDP port, for membership communications (gossip) -->
<ControlPort>7001</ControlPort>
<!--
~ The address to bind the Thrift RPC service to. Unlike ListenAddress
~ above, you *can* specify 0.0.0.0 here if you want Thrift to listen on
~ all interfaces.
~
~ Leaving this blank has the same effect it does for ListenAddress,
~ (i.e. it will be based on the configured hostname of the node).
-->
<ThriftAddress>localhost</ThriftAddress>
<!-- Thrift RPC port (the port clients connect to). -->
<ThriftPort>9160</ThriftPort>
<!--
~ Whether or not to use a framed transport for Thrift. If this option
~ is set to true then you must also use a framed transport on the
~ client-side, (framed and non-framed transports are not compatible).
-->
<ThriftFramedTransport>false</ThriftFramedTransport>
<!--======================================================================-->
<!-- Memory, Disk, and Performance -->
<!--======================================================================-->
<!--
~ Buffer size to use when performing contiguous column slices. Increase
~ this to the size of the column slices you typically perform.
~ (Name-based queries are performed with a buffer size of
~ ColumnIndexSizeInKB.)
-->
<SlicedBufferSizeInKB>64</SlicedBufferSizeInKB>
<!--
~ Buffer size to use when flushing memtables to disk. (Only one
~ memtable is ever flushed at a time.) Increase (decrease) the index
~ buffer size relative to the data buffer if you have few (many)
~ columns per key. Bigger is only better _if_ your memtables get large
~ enough to use the space. (Check in your data directory after your
~ app has been running long enough.) -->
<FlushDataBufferSizeInMB>32</FlushDataBufferSizeInMB>
<FlushIndexBufferSizeInMB>8</FlushIndexBufferSizeInMB>
<!--
~ Add column indexes to a row after its contents reach this size.
~ Increase if your column values are large, or if you have a very large
~ number of columns. The competing causes are, Cassandra has to
~ deserialize this much of the row to read a single column, so you want
~ it to be small - at least if you do many partial-row reads - but all
~ the index data is read for each access, so you don't want to generate
~ that wastefully either.
-->
<ColumnIndexSizeInKB>64</ColumnIndexSizeInKB>
<!--
~ The maximum amount of data to store in memory per ColumnFamily before
~ flushing to disk. Note: There is one memtable per column family, and
~ this threshold is based solely on the amount of data stored, not
~ actual heap memory usage (there is some overhead in indexing the
~ columns).
-->
<MemtableSizeInMB>64</MemtableSizeInMB>
<!--
~ The maximum number of columns in millions to store in memory per
~ ColumnFamily before flushing to disk. This is also a per-memtable
~ setting. Use with MemtableSizeInMB to tune memory usage.
-->
<MemtableObjectCountInMillions>0.1</MemtableObjectCountInMillions>
<!--
~ The maximum time to leave a dirty memtable unflushed.
~ (While any affected columnfamilies have unflushed data from a
~ commit log segment, that segment cannot be deleted.)
~ This needs to be large enough that it won't cause a flush storm
~ of all your memtables flushing at once because none has hit
~ the size or count thresholds yet. For production, a larger
~ value such as 1440 is recommended.
-->
<MemtableFlushAfterMinutes>60</MemtableFlushAfterMinutes>
<!--
~ Unlike most systems, in Cassandra writes are faster than reads, so
~ you can afford more of those in parallel. A good rule of thumb is 2
~ concurrent reads per processor core. Increase ConcurrentWrites to
~ the number of clients writing at once if you enable CommitLogSync +
~ CommitLogSyncDelay. -->
<ConcurrentReads>8</ConcurrentReads>
<ConcurrentWrites>32</ConcurrentWrites>
<!--
~ CommitLogSync may be either "periodic" or "batch." When in batch
~ mode, Cassandra won't ack writes until the commit log has been
~ fsynced to disk. It will wait up to CommitLogSyncBatchWindowInMS
~ milliseconds for other writes, before performing the sync.
~ This is less necessary in Cassandra than in traditional databases
~ since replication reduces the odds of losing data from a failure
~ after writing the log entry but before it actually reaches the disk.
~ So the other option is "timed," where writes may be acked immediately
~ and the CommitLog is simply synced every CommitLogSyncPeriodInMS
~ milliseconds.
-->
<CommitLogSync>periodic</CommitLogSync>
<!--
~ Interval at which to perform syncs of the CommitLog in periodic mode.
~ Usually the default of 10000ms is fine; increase it if your i/o
~ load is such that syncs are taking excessively long times.
-->
<CommitLogSyncPeriodInMS>10000</CommitLogSyncPeriodInMS>
<!--
~ Delay (in milliseconds) during which additional commit log entries
~ may be written before fsync in batch mode. This will increase
~ latency slightly, but can vastly improve throughput where there are
~ many writers. Set to zero to disable (each entry will be synced
~ individually). Reasonable values range from a minimal 0.1 to 10 or
~ even more if throughput matters more than latency.
-->
<!-- <CommitLogSyncBatchWindowInMS>1</CommitLogSyncBatchWindowInMS> -->
<!--
~ Time to wait before garbage-collection deletion markers. Set this to
~ a large enough value that you are confident that the deletion marker
~ will be propagated to all replicas by the time this many seconds has
~ elapsed, even in the face of hardware failures. The default value is
~ ten days.
-->
<GCGraceSeconds>864000</GCGraceSeconds>
<!--
~ The threshold size in megabytes the binary memtable must grow to,
~ before it's submitted for flushing to disk.
-->
<BinaryMemtableSizeInMB>256</BinaryMemtableSizeInMB>
</Storage>

View file

@ -1,167 +0,0 @@
package akka.persistence.cassandra
import akka.actor.{Actor, ActorRef}
import Actor._
import akka.stm._
import org.junit.Test
import org.junit.Assert._
import org.junit.Before
import org.scalatest.junit.JUnitSuite
case class GetMapState(key: String)
case object GetVectorState
case object GetVectorSize
case object GetRefState
case class SetMapState(key: String, value: String)
case class SetVectorState(key: String)
case class SetRefState(key: String)
case class Success(key: String, value: String)
case class Failure(key: String, value: String)
case class SetMapStateOneWay(key: String, value: String)
case class SetVectorStateOneWay(key: String)
case class SetRefStateOneWay(key: String)
case class SuccessOneWay(key: String, value: String)
case class FailureOneWay(key: String, value: String)
class CassandraPersistentActor extends Actor {
self.timeout = 100000
private val mapState = CassandraStorage.newMap
private val vectorState = CassandraStorage.newVector
private val refState = CassandraStorage.newRef
def receive = { case message => atomic { atomicReceive(message) } }
def atomicReceive: Receive = {
case GetMapState(key) =>
self.reply(mapState.get(key.getBytes("UTF-8")).get)
case GetVectorSize =>
self.reply(vectorState.length.asInstanceOf[AnyRef])
case GetRefState =>
self.reply(refState.get.get)
case SetMapState(key, msg) =>
mapState.put(key.getBytes("UTF-8"), msg.getBytes("UTF-8"))
self.reply(msg)
case SetVectorState(msg) =>
vectorState.add(msg.getBytes("UTF-8"))
self.reply(msg)
case SetRefState(msg) =>
refState.swap(msg.getBytes("UTF-8"))
self.reply(msg)
case Success(key, msg) =>
mapState.put(key.getBytes("UTF-8"), msg.getBytes("UTF-8"))
vectorState.add(msg.getBytes("UTF-8"))
refState.swap(msg.getBytes("UTF-8"))
self.reply(msg)
case Failure(key, msg) =>
mapState.put(key.getBytes("UTF-8"), msg.getBytes("UTF-8"))
vectorState.add(msg.getBytes("UTF-8"))
refState.swap(msg.getBytes("UTF-8"))
fail
self.reply(msg)
}
def fail = throw new RuntimeException("Expected exception; to test fault-tolerance")
}
class CassandraPersistentActorSpec extends JUnitSuite {
// @Before
// def startCassandra = EmbeddedCassandraService.start
@Test
def testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
val stateful = actorOf[CassandraPersistentActor]
stateful.start
stateful !! SetMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "init") // set init state
stateful !! Success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state") // transactionrequired
val result = (stateful !! GetMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess")).as[Array[Byte]].get
assertEquals("new state", new String(result, 0, result.length, "UTF-8"))
}
@Test
def testMapShouldRollbackStateForStatefulServerInCaseOfFailure = {
val stateful = actorOf[CassandraPersistentActor]
stateful.start
stateful !! SetMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure", "init") // set init state
try {
stateful !! Failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state") // call failing transactionrequired method
fail("should have thrown an exception")
} catch {case e: RuntimeException => {}}
val result = (stateful !! GetMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure")).as[Array[Byte]].get
assertEquals("init", new String(result, 0, result.length, "UTF-8")) // check that state is == init state
}
@Test
def testVectorShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
val stateful = actorOf[CassandraPersistentActor]
stateful.start
stateful !! SetVectorState("init") // set init state
stateful !! Success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state") // transactionrequired
assertEquals(2, (stateful !! GetVectorSize).get.asInstanceOf[java.lang.Integer].intValue)
}
@Test
def testVectorShouldRollbackStateForStatefulServerInCaseOfFailure = {
val stateful = actorOf[CassandraPersistentActor]
stateful.start
stateful !! SetVectorState("init") // set init state
try {
stateful !! Failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state") // call failing transactionrequired method
fail("should have thrown an exception")
} catch {case e: RuntimeException => {}}
assertEquals(1, (stateful !! GetVectorSize).get.asInstanceOf[java.lang.Integer].intValue)
}
@Test
def testRefShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
val stateful = actorOf[CassandraPersistentActor]
stateful.start
stateful !! SetRefState("init") // set init state
stateful !! Success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state") // transactionrequired
val result = (stateful !! GetRefState).as[Array[Byte]].get
assertEquals("new state", new String(result, 0, result.length, "UTF-8"))
}
@Test
def testRefShouldRollbackStateForStatefulServerInCaseOfFailure = {
val stateful = actorOf[CassandraPersistentActor]
stateful.start
stateful !! SetRefState("init") // set init state
try {
stateful !! Failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state") // call failing transactionrequired method
fail("should have thrown an exception")
} catch {case e: RuntimeException => {}}
val result = (stateful !! GetRefState).as[Array[Byte]].get
assertEquals("init", new String(result, 0, result.length, "UTF-8")) // check that state is == init state
}
}
/*
object EmbeddedCassandraService {
import org.apache.cassandra.thrift.CassandraDaemon
System.setProperty("storage-config", "src/test/resources");
val cassandra = new Runnable {
val cassandraDaemon = new CassandraDaemon
cassandraDaemon.init(null)
def run = cassandraDaemon.start
}
// spawn cassandra in a new thread
val t = new Thread(cassandra)
t.setDaemon(true)
t.start
def start(): Unit = {}
}
*/

View file

@ -1,51 +0,0 @@
package akka.persistence.cassandra
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import akka.persistence.common.{QueueStorageBackendTest, VectorStorageBackendTest, MapStorageBackendTest, RefStorageBackendTest}
@RunWith(classOf[JUnitRunner])
class CassandraRefStorageBackendTestIntegration extends RefStorageBackendTest {
def dropRefs = {
CassandraStorageBackend.refAccess.drop
}
def storage = CassandraStorageBackend
}
@RunWith(classOf[JUnitRunner])
class CassandraMapStorageBackendTestIntegration extends MapStorageBackendTest {
def dropMaps = {
CassandraStorageBackend.mapAccess.drop
}
def storage = CassandraStorageBackend
}
@RunWith(classOf[JUnitRunner])
class CassandraVectorStorageBackendTestIntegration extends VectorStorageBackendTest {
def dropVectors = {
CassandraStorageBackend.vectorAccess.drop
}
def storage = CassandraStorageBackend
}
@RunWith(classOf[JUnitRunner])
class CassandraQueueStorageBackendTestIntegration extends QueueStorageBackendTest {
def dropQueues = {
CassandraStorageBackend.queueAccess.drop
}
def storage = CassandraStorageBackend
}

View file

@ -1,23 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.persistence.cassandra
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import akka.persistence.common._
@RunWith(classOf[JUnitRunner])
class CassandraTicket343TestIntegration extends Ticket343Test {
def dropMapsAndVectors: Unit = {
CassandraStorageBackend.vectorAccess.drop
CassandraStorageBackend.mapAccess.drop
}
def getVector: (String) => PersistentVector[Array[Byte]] = CassandraStorage.getVector
def getMap: (String) => PersistentMap[Array[Byte], Array[Byte]] = CassandraStorage.getMap
}

View file

@ -1,741 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.persistence.common
import akka.util.Logging
import java.lang.String
import java.nio.ByteBuffer
import collection.Map
import java.util.{Map => JMap}
import akka.persistence.common.PersistentMapBinary.COrdering._
import collection.immutable._
import collection.mutable.ArrayBuffer
private[akka] trait CommonStorageBackendAccess {
import CommonStorageBackend._
/*abstract*/
def get(owner: String, key: Array[Byte], default: Array[Byte]): Array[Byte]
def getAll(owner: String, keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] = {
keys.foldLeft(new HashMap[Array[Byte], Array[Byte]]) {
(map, key) => {
Option(get(owner, key)) match {
case Some(value) => map + (key -> value)
case None => map
}
}
}
}
def put(owner: String, key: Array[Byte], value: Array[Byte]): Unit
def putAll(owner: String, keyValues: Iterable[(Array[Byte], Array[Byte])]): Unit = {
keyValues.foreach{
kv => kv match {
case (key, value) => put(owner, key, value)
}
}
}
def delete(owner: String, key: Array[Byte]): Unit
def deleteAll(owner: String, keys: Iterable[Array[Byte]]): Unit = {
keys.foreach(delete(owner, _))
}
def drop(): Unit
/*concrete*/
def decodeMapKey(owner: String, key: Array[Byte]): Array[Byte] = key
def encodeMapKey(owner: String, key: Array[Byte]): Array[Byte] = key
def decodeIndexedKey(owner: String, key: Array[Byte]): Int = IntSerializer.fromBytes(key)
def encodeIndexedKey(owner: String, keyint: Int): Array[Byte] = IntSerializer.toBytes(keyint)
def deleteIndexed(owner: String, index: Int): Unit = delete(owner, encodeIndexedKey(owner, index))
def getIndexed(owner: String, index: Int): Array[Byte] = get(owner, encodeIndexedKey(owner, index))
def get(owner: String, key: Array[Byte]): Array[Byte] = get(owner, key, null)
def putIndexed(owner: String, index: Int, value: Array[Byte]): Unit = put(owner, encodeIndexedKey(owner, index), value)
def putAllIndexed(owner: String, values: Iterable[(Int, Array[Byte])]): Unit = {
putAll(owner, values.map{
iv => {
iv match {
case (i, value) => (encodeIndexedKey(owner, i) -> value)
}
}
})
}
def getAllIndexed(owner: String, keys: Iterable[Int]): Map[Int, Array[Byte]] = {
val byteKeys = keys.map(encodeIndexedKey(owner, _))
getAll(owner, byteKeys).map{
kv => kv match {
case (key, value) => (decodeIndexedKey(owner, key) -> value)
}
}
}
def deleteAllIndexed(owner: String, keys: Iterable[Int]): Unit = {
val byteKeys = keys.map(encodeIndexedKey(owner, _))
deleteAll(owner, byteKeys)
}
}
private[akka] trait KVStorageBackendAccess extends CommonStorageBackendAccess with Logging {
import CommonStorageBackend._
import KVStorageBackend._
def put(key: Array[Byte], value: Array[Byte]): Unit
def get(key: Array[Byte]): Array[Byte]
def get(key: Array[Byte], default: Array[Byte]): Array[Byte]
def getAll(keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]]
def delete(key: Array[Byte]): Unit
override def decodeMapKey(owner: String, key: Array[Byte]): Array[Byte] = {
val mapKeyLength = key.length - IntSerializer.bytesPerInt - owner.getBytes("UTF-8").length
val mapkey = new Array[Byte](mapKeyLength)
System.arraycopy(key, key.length - mapKeyLength, mapkey, 0, mapKeyLength)
mapkey
}
override def decodeIndexedKey(owner: String, key: Array[Byte]): Int = {
IntSerializer.fromBytes(decodeMapKey(owner,key))
}
override def put(owner: String, key: Array[Byte], value: Array[Byte]): Unit = {
put(getKey(owner, key), value)
}
override def putIndexed(owner: String, index: Int, value: Array[Byte]): Unit = {
put(getIndexedKey(owner, index), value)
}
override def get(owner: String, key: Array[Byte]): Array[Byte] = {
get(getKey(owner, key))
}
override def getIndexed(owner: String, index: Int): Array[Byte] = {
get(getIndexedKey(owner, index))
}
override def get(owner: String, key: Array[Byte], default: Array[Byte]): Array[Byte] = {
get(getKey(owner, key), default)
}
override def getAll(owner: String, keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] = {
getAll(keys.map{
getKey(owner, _)
})
}
override def deleteIndexed(owner: String, index: Int): Unit = {
delete(getIndexedKey(owner, index))
}
override def delete(owner: String, key: Array[Byte]): Unit = {
delete(getKey(owner, key))
}
}
private[akka] object CommonStorageBackendAccess {
implicit def stringToByteArray(st: String): Array[Byte] = {
st.getBytes("UTF-8")
}
}
private[akka] object CommonStorageBackend {
val nullMapValueHeader = 0x00.byteValue
val nullMapValue: Array[Byte] = Array(nullMapValueHeader)
val notNullMapValueHeader: Byte = 0xff.byteValue
val mapKeySetKeyHeader = 0x00.byteValue
val mapKeyHeader = 0xff.byteValue
val mapKeysIndex: Array[Byte] = new Array[Byte](1).padTo(1, mapKeySetKeyHeader)
val mapKeysWrapperPad: Array[Byte] = new Array[Byte](1).padTo(1, mapKeyHeader)
/**
* Wrap map key prepends mapKeysWrapperPad (1-byte) to map keys so that we can
* use a seperate 1 byte key to store the map keyset.
*
* This basically creates the map key used in underlying storage
*/
def wrapMapKey(key: Array[Byte]): Array[Byte] = {
val wrapped = new Array[Byte](key.length + mapKeysWrapperPad.length)
System.arraycopy(mapKeysWrapperPad, 0, wrapped, 0, mapKeysWrapperPad.length)
System.arraycopy(key, 0, wrapped, mapKeysWrapperPad.length, key.length)
wrapped
}
/**
* unwrapMapKey removes the mapKeysWrapperPad, this translates the map key used
* in underlying storage back to a key that is understandable by the frontend
*/
def unwrapMapKey(key: Array[Byte]): Array[Byte] = {
val unwrapped = new Array[Byte](key.length - mapKeysWrapperPad.length)
System.arraycopy(key, mapKeysWrapperPad.length, unwrapped, 0, unwrapped.length)
unwrapped
}
def getStoredMapValue(value: Array[Byte]): Array[Byte] = {
value match {
case null => nullMapValue
case value => {
val stored = new Array[Byte](value.length + 1)
stored(0) = notNullMapValueHeader
System.arraycopy(value, 0, stored, 1, value.length)
stored
}
}
}
def getMapValueFromStored(value: Array[Byte]): Array[Byte] = {
if (value(0) == nullMapValueHeader) {
null
} else if (value(0) == notNullMapValueHeader) {
val returned = new Array[Byte](value.length - 1)
System.arraycopy(value, 1, returned, 0, value.length - 1)
returned
} else {
throw new StorageException("unknown header byte on map value:" + value(0))
}
}
object IntSerializer {
val bytesPerInt = java.lang.Integer.SIZE / java.lang.Byte.SIZE
def toBytes(i: Int) = ByteBuffer.wrap(new Array[Byte](bytesPerInt)).putInt(i).array()
def fromBytes(bytes: Array[Byte]) = ByteBuffer.wrap(bytes).getInt()
def toString(obj: Int) = obj.toString
def fromString(str: String) = str.toInt
}
object SortedSetSerializer {
def toBytes(set: SortedSet[Array[Byte]]): Array[Byte] = {
val length = set.foldLeft(0) {
(total, bytes) => {
total + bytes.length + IntSerializer.bytesPerInt
}
}
val allBytes = new Array[Byte](length)
val written = set.foldLeft(0) {
(total, bytes) => {
val sizeBytes = IntSerializer.toBytes(bytes.length)
System.arraycopy(sizeBytes, 0, allBytes, total, sizeBytes.length)
System.arraycopy(bytes, 0, allBytes, total + sizeBytes.length, bytes.length)
total + sizeBytes.length + bytes.length
}
}
require(length == written, "Bytes Written Did not equal Calculated Length, written %d, length %d".format(written, length))
allBytes
}
def fromBytes(bytes: Array[Byte]): SortedSet[Array[Byte]] = {
var set = new TreeSet[Array[Byte]]
if (bytes.length > IntSerializer.bytesPerInt) {
var pos = 0
while (pos < bytes.length) {
val lengthBytes = new Array[Byte](IntSerializer.bytesPerInt)
System.arraycopy(bytes, pos, lengthBytes, 0, IntSerializer.bytesPerInt)
pos += IntSerializer.bytesPerInt
val length = IntSerializer.fromBytes(lengthBytes)
val item = new Array[Byte](length)
System.arraycopy(bytes, pos, item, 0, length)
set = set + item
pos += length
}
}
set
}
}
}
private[akka] object KVStorageBackend {
import CommonStorageBackend._
/**
* Concat the ownerlenght+owner+key+ of owner so owned data will be colocated
* Store the length of owner as first byte to work around the rare case
* where ownerbytes1 + keybytes1 == ownerbytes2 + keybytes2 but ownerbytes1 != ownerbytes2
*/
def getKey(owner: String, key: Array[Byte]): Array[Byte] = {
val ownerBytes: Array[Byte] = owner.getBytes("UTF-8")
val ownerLenghtBytes: Array[Byte] = IntSerializer.toBytes(owner.length)
val theKey = new Array[Byte](ownerLenghtBytes.length + ownerBytes.length + key.length)
System.arraycopy(ownerLenghtBytes, 0, theKey, 0, ownerLenghtBytes.length)
System.arraycopy(ownerBytes, 0, theKey, ownerLenghtBytes.length, ownerBytes.length)
System.arraycopy(key, 0, theKey, ownerLenghtBytes.length + ownerBytes.length, key.length)
theKey
}
def getIndexedKey(owner: String, index: Int): Array[Byte] = {
getKey(owner, IntSerializer.toBytes(index))
}
}
private[akka] trait CommonStorageBackend extends MapStorageBackend[Array[Byte], Array[Byte]] with VectorStorageBackend[Array[Byte]] with RefStorageBackend[Array[Byte]] with QueueStorageBackend[Array[Byte]] with Logging {
import CommonStorageBackend._
val vectorHeadIndex = -1
val vectorTailIndex = -2
val queueHeadIndex = -1
val queueTailIndex = -2
val zero = IntSerializer.toBytes(0)
val refItem = "refItem".getBytes("UTF-8")
implicit val ordering = ArrayOrdering
def refAccess: CommonStorageBackendAccess
def vectorAccess: CommonStorageBackendAccess
def mapAccess: CommonStorageBackendAccess
def queueAccess: CommonStorageBackendAccess
def getRefStorageFor(name: String): Option[Array[Byte]] = {
val result: Array[Byte] = refAccess.get(name, refItem)
Option(result)
}
def insertRefStorageFor(name: String, element: Array[Byte]) = {
element match {
case null => refAccess.delete(name, refItem)
case _ => refAccess.put(name, refItem, element)
}
}
def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = {
val allkeys: SortedSet[Array[Byte]] = getMapKeys(name)
val range = allkeys.rangeImpl(start, finish).take(count)
getKeyValues(name, range)
}
def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = {
val keys = getMapKeys(name)
getKeyValues(name, keys)
}
private def getKeyValues(name: String, keys: SortedSet[Array[Byte]]): List[(Array[Byte], Array[Byte])] = {
val all: Map[Array[Byte], Array[Byte]] =
mapAccess.getAll(name, keys)
var returned = new TreeMap[Array[Byte], Array[Byte]]()(ordering)
all.foreach{
(entry) => {
entry match {
case (namePlusKey: Array[Byte], value: Array[Byte]) => {
//need to fix here
returned += mapAccess.decodeMapKey(name, unwrapMapKey(namePlusKey)) -> getMapValueFromStored(value)
}
}
}
}
returned.toList
}
def getMapStorageSizeFor(name: String): Int = {
val keys = getMapKeys(name)
keys.size
}
def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = {
val result: Array[Byte] = mapAccess.get(name, wrapMapKey(key))
result match {
case null => None
case _ => Some(getMapValueFromStored(result))
}
}
def removeMapStorageFor(name: String, key: Array[Byte]) = {
val wrapped = wrapMapKey(key)
var keys = getMapKeys(name)
keys -= wrapped
putMapKeys(name, keys)
mapAccess.delete(name, wrapped)
}
def removeMapStorageFor(name: String) = {
val keys = getMapKeys(name)
keys.foreach{
key =>
mapAccess.delete(name, key)
}
mapAccess.delete(name, mapKeysIndex)
}
def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]) = {
val wrapped = wrapMapKey(key)
mapAccess.put(name, wrapped, getStoredMapValue(value))
var keys = getMapKeys(name)
keys += wrapped
putMapKeys(name, keys)
}
def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) = {
val toInsert = entries.map{
kv => kv match {
case (key, value) => (wrapMapKey(key) -> getStoredMapValue(value))
}
}
mapAccess.putAll(name, toInsert)
val newKeys = toInsert.map{
case (key, value) => {
key
}
}
var keys = getMapKeys(name)
keys ++= newKeys
putMapKeys(name, keys)
}
def putMapKeys(name: String, keys: SortedSet[Array[Byte]]) = {
mapAccess.put(name, mapKeysIndex, SortedSetSerializer.toBytes(keys))
}
def getMapKeys(name: String): SortedSet[Array[Byte]] = {
SortedSetSerializer.fromBytes(mapAccess.get(name, mapKeysIndex, Array.empty[Byte]))
}
def getVectorStorageSizeFor(name: String): Int = {
getVectorMetadata(name).size
}
def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = {
val mdata = getVectorMetadata(name)
val st = start.getOrElse(0)
var cnt =
if (finish.isDefined) {
val f = finish.get
if (f >= st) (f - st) else count
} else {
count
}
if (cnt > (mdata.size - st)) {
cnt = mdata.size - st
}
val indexes = mdata.getRangeIndexes(st, count)
val result = vectorAccess.getAllIndexed(name, indexes)
indexes.map(result.get(_).get).toList
}
def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = {
val mdata = getVectorMetadata(name)
if (mdata.size > 0 && index < mdata.size) {
vectorAccess.getIndexed(name, mdata.getRangeIndexes(index, 1)(0))
} else {
throw new StorageException("In Vector:" + name + " No such Index:" + index)
}
}
def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) = {
val mdata = getVectorMetadata(name)
if (mdata.size > 0 && index < mdata.size) {
elem match {
case null => vectorAccess.deleteIndexed(name, mdata.getRangeIndexes(index, 1)(0))
case _ => vectorAccess.putIndexed(name, mdata.getRangeIndexes(index, 1)(0), elem)
}
} else {
throw new StorageException("In Vector:" + name + " No such Index:" + index)
}
}
def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) = {
var mdata = getVectorMetadata(name)
var deletes: List[Int] = Nil
var puts: List[(Int, Array[Byte])] = Nil
elements.foreach{
element => {
if (mdata.canInsert) {
element match {
case null => deletes = mdata.head :: deletes
case _ => puts = (mdata.head -> element) :: puts
}
mdata = mdata.copy(head = mdata.nextInsert)
} else {
throw new IllegalStateException("The vector dosent have enough capacity to insert these entries")
}
}
}
vectorAccess.deleteAllIndexed(name, deletes)
vectorAccess.putAllIndexed(name, puts)
vectorAccess.putIndexed(name, vectorHeadIndex, IntSerializer.toBytes(mdata.head))
}
def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = {
val mdata = getVectorMetadata(name)
if (mdata.canInsert) {
element match {
case null => vectorAccess.deleteIndexed(name, mdata.head)
case _ => vectorAccess.putIndexed(name, mdata.head, element)
}
vectorAccess.putIndexed(name, vectorHeadIndex, IntSerializer.toBytes(mdata.nextInsert))
} else {
throw new IllegalStateException("The vector %s is full".format(name))
}
}
override def removeVectorStorageEntryFor(name: String) = {
val mdata = getVectorMetadata(name)
if (mdata.canRemove) {
vectorAccess.putIndexed(name, vectorTailIndex, IntSerializer.toBytes(mdata.nextRemove))
try
{
vectorAccess.deleteIndexed(name, mdata.tail)
} catch {
case e: Exception => log.warn("Exception while trying to clean up a popped element from the vector, this is acceptable")
}
} else {
//blow up or not?
}
}
def getVectorMetadata(name: String): VectorMetadata = {
val result = vectorAccess.getAllIndexed(name, List(vectorHeadIndex, vectorTailIndex))
val head = result.getOrElse(vectorHeadIndex, zero)
val tail = result.getOrElse(vectorTailIndex, zero)
val mdata = VectorMetadata(IntSerializer.fromBytes(head), IntSerializer.fromBytes(tail))
mdata
}
def getOrDefaultToZero(map: Map[Array[Byte], Array[Byte]], key: Array[Byte]): Int = {
map.get(key) match {
case Some(value) => IntSerializer.fromBytes(value)
case None => 0
}
}
def remove(name: String): Boolean = {
val mdata = getQueueMetadata(name)
mdata.getActiveIndexes foreach {
index =>
queueAccess.deleteIndexed(name, index)
}
queueAccess.deleteIndexed(name, queueHeadIndex)
queueAccess.deleteIndexed(name, queueTailIndex)
true
}
def peek(name: String, start: Int, count: Int): List[Array[Byte]] = {
val mdata = getQueueMetadata(name)
val indexes = mdata.getPeekIndexes(start, count)
val result = queueAccess.getAllIndexed(name, indexes)
indexes.map(result.get(_).get).toList
}
def size(name: String): Int = {
getQueueMetadata(name).size
}
def dequeue(name: String): Option[Array[Byte]] = {
val mdata = getQueueMetadata(name)
if (mdata.canDequeue) {
try
{
val dequeued = queueAccess.getIndexed(name, mdata.head)
queueAccess.putIndexed(name, queueHeadIndex, IntSerializer.toBytes(mdata.nextDequeue))
Some(dequeued)
} finally {
try
{
queueAccess.deleteIndexed(name, mdata.head)
} catch {
//a failure to delete is ok, just leaves a K-V in Voldemort that will be overwritten if the queue ever wraps around
case e: Exception => log.warn(e, "caught an exception while deleting a dequeued element, however this will not cause any inconsistency in the queue")
}
}
} else {
None
}
}
def enqueue(name: String, item: Array[Byte]): Option[Int] = {
val mdata = getQueueMetadata(name)
if (mdata.canEnqueue) {
item match {
case null => queueAccess.deleteIndexed(name, mdata.tail)
case _ => queueAccess.putIndexed(name, mdata.tail, item)
}
queueAccess.putIndexed(name, queueTailIndex, IntSerializer.toBytes(mdata.nextEnqueue))
Some(mdata.size + 1)
} else {
None
}
}
def getQueueMetadata(name: String): QueueMetadata = {
val result = queueAccess.getAllIndexed(name, List(vectorHeadIndex, vectorTailIndex))
val head = result.get(vectorHeadIndex).getOrElse(zero)
val tail = result.get(vectorTailIndex).getOrElse(zero)
QueueMetadata(IntSerializer.fromBytes(head), IntSerializer.fromBytes(tail))
}
//wrapper for null
case class QueueMetadata(head: Int, tail: Int) {
//queue is an sequence with indexes from 0 to Int.MAX_VALUE
//wraps around when one pointer gets to max value
//head has an element in it.
//tail is the next slot to write to.
def size = {
if (tail >= head) {
tail - head
} else {
//queue has wrapped
(Integer.MAX_VALUE - head) + (tail + 1)
}
}
def canEnqueue = {
//the -1 stops the tail from catching the head on a wrap around
size < Integer.MAX_VALUE - 1
}
def canDequeue = {
size > 0
}
def getActiveIndexes(): IndexedSeq[Int] = {
if (tail >= head) {
Range(head, tail)
} else {
//queue has wrapped
val headRange = Range.inclusive(head, Integer.MAX_VALUE)
(if (tail > 0) {
headRange ++ Range(0, tail)
} else {
headRange
})
}
}
def getPeekIndexes(start: Int, count: Int): IndexedSeq[Int] = {
val indexes = getActiveIndexes
if (indexes.size < start) {
IndexedSeq.empty[Int]
} else {
indexes.drop(start).take(count)
}
}
def nextEnqueue = {
tail match {
case Integer.MAX_VALUE => 0
case _ => tail + 1
}
}
def nextDequeue = {
head match {
case Integer.MAX_VALUE => 0
case _ => head + 1
}
}
}
case class VectorMetadata(head: Int, tail: Int) {
def size = {
if (head >= tail) {
head - tail
} else {
//queue has wrapped
(Integer.MAX_VALUE - tail) + (head + 1)
}
}
def canInsert = {
//the -1 stops the tail from catching the head on a wrap around
size < Integer.MAX_VALUE - 1
}
def canRemove = {
size > 0
}
def getActiveIndexes(): IndexedSeq[Int] = {
if (head >= tail) {
Range(tail, head)
} else {
//queue has wrapped
val headRange = Range.inclusive(tail, Integer.MAX_VALUE)
(if (head > 0) {
headRange ++ Range(0, head)
} else {
headRange
})
}
}
def getRangeIndexes(start: Int, count: Int): IndexedSeq[Int] = {
val indexes = getActiveIndexes.reverse
if (indexes.size < start) {
IndexedSeq.empty[Int]
} else {
indexes.drop(start).take(count)
}
}
def nextInsert = {
head match {
case Integer.MAX_VALUE => 0
case _ => head + 1
}
}
def nextRemove = {
tail match {
case Integer.MAX_VALUE => 0
case _ => tail + 1
}
}
}
}

View file

@ -1,91 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.persistence.common
import org.apache.commons.pool._
import org.apache.commons.pool.impl._
import org.apache.thrift.transport._
trait Pool[T] extends java.io.Closeable {
def borrowObject: T
def returnObject(t: T): Unit
def invalidateObject(t: T): Unit
def addObject(): Unit
def getNumIdle: Int
def getNumActive: Int
def clear(): Unit
def setFactory(factory: PoolItemFactory[T]): Unit
}
trait PoolFactory[T] {
def createPool: Pool[T]
}
trait PoolItemFactory[T] {
def makeObject: T
def destroyObject(t: T): Unit
def validateObject(t: T): Boolean
def activateObject(t: T): Unit
def passivateObject(t: T): Unit
}
trait PoolBridge[T, OP <: ObjectPool] extends Pool[T] {
val impl: OP
override def borrowObject: T = impl.borrowObject.asInstanceOf[T]
override def returnObject(t: T) = impl.returnObject(t)
override def invalidateObject(t: T) = impl.invalidateObject(t)
override def addObject = impl.addObject
override def getNumIdle: Int = impl.getNumIdle
override def getNumActive: Int = impl.getNumActive
override def clear(): Unit = impl.clear()
override def close(): Unit = impl.close()
override def setFactory(factory: PoolItemFactory[T]) = impl.setFactory(toPoolableObjectFactory(factory))
def toPoolableObjectFactory[T](pif: PoolItemFactory[T]) = new PoolableObjectFactory {
def makeObject: Object = pif.makeObject.asInstanceOf[Object]
def destroyObject(o: Object): Unit = pif.destroyObject(o.asInstanceOf[T])
def validateObject(o: Object): Boolean = pif.validateObject(o.asInstanceOf[T])
def activateObject(o: Object): Unit = pif.activateObject(o.asInstanceOf[T])
def passivateObject(o: Object): Unit = pif.passivateObject(o.asInstanceOf[T])
}
}
object StackPool {
def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,StackObjectPool] {
val impl = new StackObjectPool(toPoolableObjectFactory(factory))
}
def apply[T](factory: PoolItemFactory[T], maxIdle: Int) = new PoolBridge[T,StackObjectPool] {
val impl = new StackObjectPool(toPoolableObjectFactory(factory),maxIdle)
}
def apply[T](factory: PoolItemFactory[T], maxIdle: Int, initIdleCapacity: Int) = new PoolBridge[T,StackObjectPool] {
val impl = new StackObjectPool(toPoolableObjectFactory(factory),maxIdle,initIdleCapacity)
}
}
object SoftRefPool {
def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,SoftReferenceObjectPool] {
val impl = new SoftReferenceObjectPool(toPoolableObjectFactory(factory))
}
}
trait TransportFactory[T <: TTransport] extends PoolItemFactory[T] {
def createTransport: T
def makeObject: T = createTransport
def destroyObject(transport: T): Unit = transport.close
def validateObject(transport: T) = transport.isOpen
def activateObject(transport: T): Unit = if( !transport.isOpen ) transport.open else ()
def passivateObject(transport: T): Unit = transport.flush
}
case class SocketProvider(val host: String, val port: Int) extends TransportFactory[TSocket] {
def createTransport = {
val t = new TSocket(host, port)
t.open
t
}
}

View file

@ -1,876 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.persistence.common
import akka.stm._
import akka.stm.TransactionManagement.transaction
import akka.util.Logging
import akka.japi.{Option => JOption}
import collection.mutable.ArraySeq
// FIXME move to 'stm' package + add message with more info
class NoTransactionInScopeException extends RuntimeException
class StorageException(message: String) extends RuntimeException(message)
/**
* Example Scala usage.
* <p/>
* New map with generated id.
* <pre>
* val myMap = CassandraStorage.newMap
* </pre>
*
* New map with user-defined id.
* <pre>
* val myMap = MongoStorage.newMap(id)
* </pre>
*
* Get map by user-defined id.
* <pre>
* val myMap = CassandraStorage.getMap(id)
* </pre>
*
* Example Java usage:
* <pre>
* PersistentMap<Object, Object> myMap = MongoStorage.newMap();
* </pre>
* Or:
* <pre>
* MongoPersistentMap myMap = MongoStorage.getMap(id);
* </pre>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
*/
trait Storage {
type ElementType
def newMap: PersistentMap[ElementType, ElementType]
def newVector: PersistentVector[ElementType]
def newRef: PersistentRef[ElementType]
def newQueue: PersistentQueue[ElementType] = // only implemented for redis
throw new UnsupportedOperationException
def newSortedSet: PersistentSortedSet[ElementType] = // only implemented for redis
throw new UnsupportedOperationException
def getMap(id: String): PersistentMap[ElementType, ElementType]
def getVector(id: String): PersistentVector[ElementType]
def getRef(id: String): PersistentRef[ElementType]
def getQueue(id: String): PersistentQueue[ElementType] = // only implemented for redis
throw new UnsupportedOperationException
def getSortedSet(id: String): PersistentSortedSet[ElementType] = // only implemented for redis
throw new UnsupportedOperationException
def newMap(id: String): PersistentMap[ElementType, ElementType]
def newVector(id: String): PersistentVector[ElementType]
def newRef(id: String): PersistentRef[ElementType]
def newQueue(id: String): PersistentQueue[ElementType] = // only implemented for redis
throw new UnsupportedOperationException
def newSortedSet(id: String): PersistentSortedSet[ElementType] = // only implemented for redis
throw new UnsupportedOperationException
}
private[akka] object PersistentMap {
// operations on the Map
sealed trait Op
case object PUT extends Op
case object REM extends Op
case object UPD extends Op
case object CLR extends Op
}
/**
* Implementation of <tt>PersistentMap</tt> for every concrete
* storage will have the same workflow. This abstracts the workflow.
*
* Subclasses just need to provide the actual concrete instance for the
* abstract val <tt>storage</tt>.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
with Transactional with Committable with Abortable with Logging {
import scalaj.collection.Imports._
def asJavaMap() : java.util.Map[K, V] = this.asJava
//Import Ops
import PersistentMap._
// append only log: records all mutating operations
protected val appendOnlyTxLog = TransactionalVector[LogEntry]()
case class LogEntry(key: Option[K], value: Option[V], op: Op)
// need to override in subclasses e.g. "sameElements" for Array[Byte]
def equal(k1: K, k2: K): Boolean = k1 == k2
// Seqable type that's required for maintaining the log of distinct keys affected in current transaction
type T <: Equals
// converts key K to the Seqable type Equals
def toEquals(k: K): T
// keys affected in the current transaction
protected val keysInCurrentTx = TransactionalMap[T, K]()
protected def addToListOfKeysInTx(key: K): Unit =
keysInCurrentTx += (toEquals(key), key)
protected def clearDistinctKeys = keysInCurrentTx.clear
protected def filterTxLogByKey(key: K): IndexedSeq[LogEntry] =
appendOnlyTxLog filter (e => e.key.map(equal(_, key)).getOrElse(true))
// need to get current value considering the underlying storage as well as the transaction log
protected def getCurrentValue(key: K): Option[V] = {
// get all mutating entries for this key for this tx
val txEntries = filterTxLogByKey(key)
// get the snapshot from the underlying store for this key
val underlying = try {
storage.getMapStorageEntryFor(uuid, key)
} catch {case e: Exception => None}
if (txEntries.isEmpty) underlying
else txEntries.last match {
case LogEntry(_, _, CLR) => None
case _ => replay(txEntries, key, underlying)
}
}
// replay all tx entries for key k with seed = initial
private def replay(txEntries: IndexedSeq[LogEntry], key: K, initial: Option[V]): Option[V] = {
import scala.collection.mutable._
val m = initial match {
case None => Map.empty[K, V]
case Some(v) => Map((key, v))
}
txEntries.foreach {
case LogEntry(k, v, o) => o match {
case PUT => m.put(k.get, v.get)
case REM => m -= k.get
case UPD => m.update(k.get, v.get)
case CLR => Map.empty[K, V]
}
}
m get key
}
// to be concretized in subclasses
val storage: MapStorageBackend[K, V]
def commit = {
appendOnlyTxLog.foreach {
case LogEntry(k, v, o) => o match {
case PUT => storage.insertMapStorageEntryFor(uuid, k.get, v.get)
case UPD => storage.insertMapStorageEntryFor(uuid, k.get, v.get)
case REM => storage.removeMapStorageFor(uuid, k.get)
case CLR => storage.removeMapStorageFor(uuid)
}
}
appendOnlyTxLog.clear
clearDistinctKeys
}
def abort = {
appendOnlyTxLog.clear
clearDistinctKeys
}
def -=(key: K) = {
remove(key)
this
}
override def +=(kv: (K, V)) = {
put(kv._1, kv._2)
this
}
def +=(key: K, value: V) = {
put(key, value)
this
}
override def put(key: K, value: V): Option[V] = {
register
val curr = getCurrentValue(key)
appendOnlyTxLog add LogEntry(Some(key), Some(value), PUT)
addToListOfKeysInTx(key)
curr
}
override def update(key: K, value: V) {
register
val curr = getCurrentValue(key)
appendOnlyTxLog add LogEntry(Some(key), Some(value), UPD)
addToListOfKeysInTx(key)
curr
}
override def remove(key: K) : Option[V] = {
register
val curr = getCurrentValue(key)
appendOnlyTxLog add LogEntry(Some(key), None, REM)
addToListOfKeysInTx(key)
curr
}
def slice(start: Option[K], count: Int): List[(K, V)] =
slice(start, None, count)
def slice(start: Option[K], finish: Option[K], count: Int): List[(K, V)]
override def clear = {
register
appendOnlyTxLog add LogEntry(None, None, CLR)
clearDistinctKeys
}
override def contains(key: K): Boolean = try {
filterTxLogByKey(key) match {
case Seq() => // current tx doesn't use this
storage.getMapStorageEntryFor(uuid, key).isDefined // check storage
case txs => // present in log
val lastOp = txs.last.op
lastOp != REM && lastOp != CLR // last entry cannot be a REM
}
} catch {case e: Exception => false}
protected def existsInStorage(key: K): Option[V] = try {
storage.getMapStorageEntryFor(uuid, key)
} catch {
case e: Exception => None
}
override def size: Int = try {
// partition key set affected in current tx into those which r added & which r deleted
val (keysAdded, keysRemoved) = keysInCurrentTx.map {
case (kseq, k) => ((kseq, k), getCurrentValue(k))
}.partition(_._2.isDefined)
// keys which existed in storage but removed in current tx
val inStorageRemovedInTx =
keysRemoved.keySet
.map(_._2)
.filter(k => existsInStorage(k).isDefined)
.size
// all keys in storage
val keysInStorage =
storage.getMapStorageFor(uuid)
.map {case (k, v) => toEquals(k)}
.toSet
// (keys that existed UNION keys added ) - (keys removed)
(keysInStorage union keysAdded.keySet.map(_._1)).size - inStorageRemovedInTx
} catch {
case e: Exception => 0
}
// get must consider underlying storage & current uncommitted tx log
override def get(key: K): Option[V] = getCurrentValue(key)
def iterator: Iterator[Tuple2[K, V]]
protected def register = {
if (transaction.get.isEmpty) throw new NoTransactionInScopeException
transaction.get.get.register("Map:" + uuid, this)
}
}
object PersistentMapBinary {
object COrdering {
//frontend
implicit object ArraySeqOrdering extends Ordering[ArraySeq[Byte]] {
def compare(o1: ArraySeq[Byte], o2: ArraySeq[Byte]) =
ArrayOrdering.compare(o1.toArray, o2.toArray)
}
//backend
implicit object ArrayOrdering extends Ordering[Array[Byte]] {
import java.lang.{Math=>M}
def compare(o1: Array[Byte], o2: Array[Byte]): Int = {
if (o1.size == o2.size) {
for (i <- 0 until o1.size) {
var a = o1(i)
var b = o2(i)
if (a != b) {
return (a - b) / (M.abs(a - b))
}
}
0
} else {
(o1.length - o2.length) / (M.max(1, M.abs(o1.length - o2.length)))
}
}
}
}
}
trait PersistentMapBinary extends PersistentMap[Array[Byte], Array[Byte]] {
import scala.collection.mutable.ArraySeq
type T = ArraySeq[Byte]
def toEquals(k: Array[Byte]) = ArraySeq(k: _*)
override def equal(k1: Array[Byte], k2: Array[Byte]): Boolean = k1 sameElements k2
import scala.collection.immutable.{TreeMap, SortedMap}
private def replayAllKeys: SortedMap[ArraySeq[Byte], Array[Byte]] = {
import PersistentMapBinary.COrdering._
// need ArraySeq for ordering
val fromStorage =
TreeMap(storage.getMapStorageFor(uuid).map {case (k, v) => (ArraySeq(k: _*), v)}: _*)
val (keysAdded, keysRemoved) = keysInCurrentTx.map {
case (_, k) => (k, getCurrentValue(k))
}.partition(_._2.isDefined)
val inStorageRemovedInTx =
keysRemoved.keySet
.filter(k => existsInStorage(k).isDefined)
.map(k => ArraySeq(k: _*))
(fromStorage -- inStorageRemovedInTx) ++ keysAdded.map {case (k, v) => (ArraySeq(k: _*), v.get)}
}
override def slice(start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = try {
val newMap = replayAllKeys
if (newMap isEmpty) List[(Array[Byte], Array[Byte])]()
val startKey =
start match {
case Some(bytes) => Some(ArraySeq(bytes: _*))
case None => None
}
val endKey =
finish match {
case Some(bytes) => Some(ArraySeq(bytes: _*))
case None => None
}
((startKey, endKey, count): @unchecked) match {
case ((Some(s), Some(e), _)) =>
newMap.range(s, e)
.toList
.map(e => (e._1.toArray, e._2))
.toList
case ((Some(s), None, c)) if c > 0 =>
newMap.from(s)
.iterator
.take(count)
.map(e => (e._1.toArray, e._2))
.toList
case ((Some(s), None, _)) =>
newMap.from(s)
.toList
.map(e => (e._1.toArray, e._2))
.toList
case ((None, Some(e), _)) =>
newMap.until(e)
.toList
.map(e => (e._1.toArray, e._2))
.toList
}
} catch {case e: Exception => Nil}
override def iterator: Iterator[(Array[Byte], Array[Byte])] = {
new Iterator[(Array[Byte], Array[Byte])] {
private var elements = replayAllKeys
override def next: (Array[Byte], Array[Byte]) = synchronized {
val (k, v) = elements.head
elements = elements.tail
(k.toArray, v)
}
override def hasNext: Boolean = synchronized {!elements.isEmpty}
}
}
/**
* Java API.
*/
def javaIterator: java.util.Iterator[java.util.Map.Entry[Array[Byte],Array[Byte]]] = {
new java.util.Iterator[java.util.Map.Entry[Array[Byte],Array[Byte]]] {
private var elements = replayAllKeys
override def next: java.util.Map.Entry[Array[Byte], Array[Byte]] = synchronized {
val (k, v) = elements.head
elements = elements.tail
val entry = new java.util.Map.Entry[Array[Byte], Array[Byte]] {
override def getKey = k.toArray
override def getValue = v
override def setValue(v: Array[Byte]) = throw new UnsupportedOperationException("Use put or update methods to change a map entry.")
}
entry
}
override def hasNext: Boolean = synchronized { !elements.isEmpty }
override def remove: Unit = throw new UnsupportedOperationException("Use remove method to remove a map entry.")
}
}
}
private[akka] object PersistentVector {
// operations on the Vector
sealed trait Op
case object ADD extends Op
case object UPD extends Op
case object POP extends Op
}
/**
* Implements a template for a concrete persistent transactional vector based storage.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committable with Abortable {
//Import Ops
import PersistentVector._
import scalaj.collection.Imports._
def asJavaList() : java.util.List[T] = this.asJava
// append only log: records all mutating operations
protected val appendOnlyTxLog = TransactionalVector[LogEntry]()
case class LogEntry(index: Option[Int], value: Option[T], op: Op)
// need to override in subclasses e.g. "sameElements" for Array[Byte]
// def equal(v1: T, v2: T): Boolean = v1 == v2
val storage: VectorStorageBackend[T]
def commit = {
for (entry <- appendOnlyTxLog) {
(entry: @unchecked) match {
case LogEntry(_, Some(v), ADD) => storage.insertVectorStorageEntryFor(uuid, v)
case LogEntry(Some(i), Some(v), UPD) => storage.updateVectorStorageEntryFor(uuid, i, v)
case LogEntry(_, _, POP) => storage.removeVectorStorageEntryFor(uuid)
}
}
appendOnlyTxLog.clear
}
def abort = {
appendOnlyTxLog.clear
}
private def replay: List[T] = {
import scala.collection.mutable.ArrayBuffer
var elemsStorage = ArrayBuffer(storage.getVectorStorageRangeFor(uuid, None, None, storage.getVectorStorageSizeFor(uuid)).reverse: _*)
for (entry <- appendOnlyTxLog) {
(entry: @unchecked) match {
case LogEntry(_, Some(v), ADD) => elemsStorage += v
case LogEntry(Some(i), Some(v), UPD) => elemsStorage.update(i, v)
case LogEntry(_, _, POP) => elemsStorage = elemsStorage.drop(1)
}
}
elemsStorage.toList.reverse
}
def +(elem: T) = add(elem)
def add(elem: T) = {
register
appendOnlyTxLog + LogEntry(None, Some(elem), ADD)
}
def apply(index: Int): T = get(index)
def get(index: Int): T = {
if (appendOnlyTxLog.isEmpty) {
storage.getVectorStorageEntryFor(uuid, index)
} else {
val curr = replay
curr(index)
}
}
override def slice(start: Int, finish: Int): IndexedSeq[T] = slice(Some(start), Some(finish))
def slice(start: Option[Int], finish: Option[Int], count: Int = 0): IndexedSeq[T] = {
val curr = replay
val s = if (start.isDefined) start.get else 0
val cnt =
if (finish.isDefined) {
val f = finish.get
if (f >= s) (f - s) else count
}
else count
if (s == 0 && cnt == 0) List().toIndexedSeq
else curr.slice(s, s + cnt).toIndexedSeq
}
/**
* Removes the <i>tail</i> element of this vector.
*/
def pop: T = {
register
val curr = replay
appendOnlyTxLog + LogEntry(None, None, POP)
curr.last
}
def update(index: Int, newElem: T) = {
register
appendOnlyTxLog + LogEntry(Some(index), Some(newElem), UPD)
}
override def first: T = get(0)
override def last: T = replay.last
def length: Int = replay.length
protected def register = {
if (transaction.get.isEmpty) throw new NoTransactionInScopeException
transaction.get.get.register("Vector" + uuid, this)
}
}
/**
* Implements a persistent reference with abstract storage.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait PersistentRef[T] extends Transactional with Committable with Abortable {
protected val ref = Ref[T]()
val storage: RefStorageBackend[T]
def commit = if (ref.isDefined) {
storage.insertRefStorageFor(uuid, ref.get)
ref.swap(null.asInstanceOf[T])
}
def abort = ref.swap(null.asInstanceOf[T])
def swap(elem: T) = {
register
ref.swap(elem)
}
def get: Option[T] = if (ref.isDefined) ref.opt else storage.getRefStorageFor(uuid)
def isDefined: Boolean = ref.isDefined || storage.getRefStorageFor(uuid).isDefined
def getOrElse(default: => T): T = {
val current = get
if (current.isDefined) current.get
else default
}
protected def register = {
if (transaction.get.isEmpty) throw new NoTransactionInScopeException
transaction.get.get.register("Ref" + uuid, this)
}
}
private[akka] object PersistentQueue {
//Operations for PersistentQueue
sealed trait QueueOp
case object ENQ extends QueueOp
case object DEQ extends QueueOp
}
/**
* Implementation of <tt>PersistentQueue</tt> for every concrete
* storage will have the same workflow. This abstracts the workflow.
* <p/>
* Enqueue is simpler, we just have to record the operation in a local
* transactional store for playback during commit. This store
* <tt>enqueueNDequeuedEntries</tt> stores the entire history of enqueue
* and dequeue that will be played at commit on the underlying store.
* </p>
* The main challenge with dequeue is that we need to return the element
* that has been dequeued. Hence in addition to the above store, we need to
* have another local queue that actually does the enqueue dequeue operations
* that take place <em>only during this transaction</em>. This gives us the
* element that will be dequeued next from the set of elements enqueued
* <em>during this transaction</em>.
* </p>
* The third item that we need is an index to the underlying storage element
* that may also have to be dequeued as part of the current transaction. This
* is modeled using a ref to an Int that points to elements in the underlyinng store.
* </p>
* Subclasses just need to provide the actual concrete instance for the
* abstract val <tt>storage</tt>.
*
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
*/
trait PersistentQueue[A] extends scala.collection.mutable.Queue[A]
with Transactional with Committable with Abortable with Logging {
//Import Ops
import PersistentQueue._
case class LogEntry(value: Option[A], op: QueueOp)
// current trail that will be played on commit to the underlying store
protected val appendOnlyTxLog = TransactionalVector[LogEntry]()
// to be concretized in subclasses
val storage: QueueStorageBackend[A]
def commit = synchronized {
for (entry <- appendOnlyTxLog) {
(entry: @unchecked) match {
case LogEntry(Some(v), ENQ) => storage.enqueue(uuid, v)
case LogEntry(_, DEQ) => storage.dequeue(uuid)
}
}
appendOnlyTxLog.clear
}
def abort = synchronized {
appendOnlyTxLog.clear
}
override def toList = replay
override def enqueue(elems: A*) = synchronized {
register
elems.foreach(e => appendOnlyTxLog.add(LogEntry(Some(e), ENQ)))
}
private def replay: List[A] = synchronized {
import scala.collection.mutable.ListBuffer
var elemsStorage = ListBuffer(storage.peek(uuid, 0, storage.size(uuid)): _*)
for (entry <- appendOnlyTxLog) {
(entry: @unchecked) match {
case LogEntry(Some(v), ENQ) => elemsStorage += v
case LogEntry(_, DEQ) => elemsStorage = elemsStorage.drop(1)
}
}
elemsStorage.toList
}
override def dequeue: A = synchronized {
register
val l = replay
if (l.isEmpty) throw new NoSuchElementException("trying to dequeue from empty queue")
appendOnlyTxLog.add(LogEntry(None, DEQ))
l.head
}
override def clear = synchronized {
register
appendOnlyTxLog.clear
}
override def size: Int = try {
replay.size
} catch {case e: Exception => 0}
override def isEmpty: Boolean = size == 0
override def +=(elem: A) = {
enqueue(elem)
this
}
def ++=(elems: Iterator[A]) = {
enqueue(elems.toList: _*)
this
}
def ++=(elems: Iterable[A]): Unit = this ++= elems.iterator
override def dequeueFirst(p: A => Boolean): Option[A] =
throw new UnsupportedOperationException("dequeueFirst not supported")
override def dequeueAll(p: A => Boolean): scala.collection.mutable.Seq[A] =
throw new UnsupportedOperationException("dequeueAll not supported")
protected def register = {
if (transaction.get.isEmpty) throw new NoTransactionInScopeException
transaction.get.get.register("Queue:" + uuid, this)
}
}
private[akka] object PersistentSortedSet {
// operations on the SortedSet
sealed trait Op
case object ADD extends Op
case object REM extends Op
}
/**
* Implements a template for a concrete persistent transactional sorted set based storage.
* <p/>
* Sorting is done based on a <i>zscore</i>. But the computation of zscore has been kept
* outside the abstraction.
* <p/>
* zscore can be implemented in a variety of ways by the calling class:
* <pre>
* trait ZScorable {
* def toZScore: Float
* }
*
* class Foo extends ZScorable {
* //.. implemnetation
* }
* </pre>
* Or we can also use views:
* <pre>
* class Foo {
* //..
* }
*
* implicit def Foo2Scorable(foo: Foo): ZScorable = new ZScorable {
* def toZScore = {
* //..
* }
* }
* </pre>
*
* and use <tt>foo.toZScore</tt> to compute the zscore and pass to the APIs.
*
* @author <a href="http://debasishg.blogspot.com"</a>
*/
trait PersistentSortedSet[A] extends Transactional with Committable with Abortable {
//Import Ops
import PersistentSortedSet._
// append only log: records all mutating operations
protected val appendOnlyTxLog = TransactionalVector[LogEntry]()
// need to override in subclasses e.g. "sameElements" for Array[Byte]
def equal(v1: A, v2: A): Boolean = v1 == v2
case class LogEntry(value: A, score: Option[Float], op: Op)
val storage: SortedSetStorageBackend[A]
def commit = {
for (entry <- appendOnlyTxLog) {
(entry: @unchecked) match {
case LogEntry(e, Some(s), ADD) => storage.zadd(uuid, String.valueOf(s), e)
case LogEntry(e, _, REM) => storage.zrem(uuid, e)
}
}
appendOnlyTxLog.clear
}
def abort = {
appendOnlyTxLog.clear
}
def +(elem: A, score: Float) = add(elem, score)
def add(elem: A, score: Float) = {
register
appendOnlyTxLog.add(LogEntry(elem, Some(score), ADD))
}
def -(elem: A) = remove(elem)
def remove(elem: A) = {
register
appendOnlyTxLog.add(LogEntry(elem, None, REM))
}
protected def replay: List[(A, Float)] = {
val es = collection.mutable.Map() ++ storage.zrangeWithScore(uuid, 0, -1)
for (entry <- appendOnlyTxLog) {
(entry: @unchecked) match {
case LogEntry(v, Some(s), ADD) => es += ((v, s))
case LogEntry(v, _, REM) => es -= v
}
}
es.toList
}
def contains(elem: A): Boolean = replay.map(_._1).contains(elem)
def size: Int = replay size
def zscore(elem: A): Float = replay.filter { case (e, s) => equal(e, elem) }.map(_._2).head
def zrange(start: Int, end: Int): List[(A, Float)] = {
import PersistentSortedSet._
// easier would have been to use a TreeSet
// problem is the treeset has to be ordered on the score
// but we cannot kick out elements with duplicate score
// But we need to treat the value (A) as set, i.e. replace duplicates with
// the latest one, as par with the behavior of redis zrange
val es = replay
// a multimap with key as A and value as Set of scores
val m = new collection.mutable.HashMap[A, collection.mutable.Set[Float]]
with collection.mutable.MultiMap[A, Float]
for(e <- es) m.addBinding(e._1, e._2)
// another list for unique values
val as = es.map(_._1).distinct
// iterate the list of unique values and for each pick the head element
// from the score map
val ts = as.map(a => (a, m(a).head)).sortWith((a, b) => a._2 < b._2)
val l = ts.size
// -1 means the last element, -2 means the second last
val s = if (start < 0) start + l else start
val e =
if (end < 0) end + l
else if (end >= l) (l - 1)
else end
// slice is open at the end, we need a closed end range
ts.iterator.slice(s, e + 1).toList
}
protected def register = {
if (transaction.get.isEmpty) throw new NoTransactionInScopeException
transaction.get.get.register("SortedSet:" + uuid, this)
}
}
trait PersistentSortedSetBinary extends PersistentSortedSet[Array[Byte]] {
import PersistentSortedSet._
override def equal(k1: Array[Byte], k2: Array[Byte]): Boolean = k1 sameElements k2
override protected def replay: List[(Array[Byte], Float)] = {
val es = collection.mutable.Map() ++ storage.zrangeWithScore(uuid, 0, -1).map { case (k, v) => (ArraySeq(k: _*), v) }
for (entry <- appendOnlyTxLog) {
(entry: @unchecked) match {
case LogEntry(v, Some(s), ADD) => es += ((ArraySeq(v: _*), s))
case LogEntry(v, _, REM) => es -= ArraySeq(v: _*)
}
}
es.toList.map { case (k, v) => (k.toArray, v) }
}
}

View file

@ -1,79 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.persistence.common
// abstracts persistence storage
trait StorageBackend
// for Maps
trait MapStorageBackend[K, V] extends StorageBackend {
def insertMapStorageEntriesFor(name: String, entries: List[Tuple2[K, V]])
def insertMapStorageEntryFor(name: String, key: K, value: V)
def removeMapStorageFor(name: String)
def removeMapStorageFor(name: String, key: K)
def getMapStorageEntryFor(name: String, key: K): Option[V]
def getMapStorageSizeFor(name: String): Int
def getMapStorageFor(name: String): List[Tuple2[K, V]]
def getMapStorageRangeFor(name: String, start: Option[K], finish: Option[K], count: Int): List[Tuple2[K, V]]
}
// for Vectors
trait VectorStorageBackend[T] extends StorageBackend {
def insertVectorStorageEntryFor(name: String, element: T)
def insertVectorStorageEntriesFor(name: String, elements: List[T])
def updateVectorStorageEntryFor(name: String, index: Int, elem: T)
def getVectorStorageEntryFor(name: String, index: Int): T
def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[T]
def getVectorStorageSizeFor(name: String): Int
def removeVectorStorageEntryFor(name:String):Unit = {
//should remove the "tail" if supported
throw new UnsupportedOperationException("VectorStorageBackend.removeVectorStorageEntry is not supported")
}
}
// for Ref
trait RefStorageBackend[T] extends StorageBackend {
def insertRefStorageFor(name: String, element: T)
def getRefStorageFor(name: String): Option[T]
}
// for Queue
trait QueueStorageBackend[T] extends StorageBackend {
// add to the end of the queue
def enqueue(name: String, item: T): Option[Int]
// pop from the front of the queue
def dequeue(name: String): Option[T]
// get the size of the queue
def size(name: String): Int
// 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[T]
// completely delete the queue
def remove(name: String): Boolean
}
trait SortedSetStorageBackend[T] extends StorageBackend {
// add item to sorted set identified by name
def zadd(name: String, zscore: String, item: T): Boolean
// remove item from sorted set identified by name
def zrem(name: String, item: T): Boolean
// cardinality of the set identified by name
def zcard(name: String): Int
// zscore of the item from sorted set identified by name
def zscore(name: String, item: T): Option[Float]
// zrange from the sorted set identified by name
def zrange(name: String, start: Int, end: Int): List[T]
// zrange with score from the sorted set identified by name
def zrangeWithScore(name: String, start: Int, end: Int): List[(T, Float)]
}

View file

@ -1,177 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package akka.persistence.common
import org.scalatest.matchers.ShouldMatchers
import akka.util.Logging
import org.scalatest.{BeforeAndAfterEach, Spec}
import scala.util.Random
import collection.immutable.{TreeMap, HashMap, HashSet}
import akka.persistence.common.PersistentMapBinary.COrdering._
/**
* Implementation Compatibility test for PersistentMap backend implementations.
*/
trait MapStorageBackendTest extends Spec with ShouldMatchers with BeforeAndAfterEach with Logging {
def storage: MapStorageBackend[Array[Byte], Array[Byte]]
def dropMaps: Unit
override def beforeEach = {
log.info("beforeEach: dropping maps")
dropMaps
}
override def afterEach = {
log.info("afterEach: dropping maps")
dropMaps
}
describe("A Properly functioning MapStorageBackend") {
it("should remove map storage properly") {
val mapName = "removeTest"
val mkey = "removeTestKey".getBytes
val value = "removeTestValue".getBytes
storage.insertMapStorageEntryFor(mapName, mkey, value)
storage.getMapStorageEntryFor(mapName, mkey).isDefined should be(true)
storage.removeMapStorageFor(mapName, mkey)
storage.getMapStorageEntryFor(mapName, mkey) should be(None)
storage.insertMapStorageEntryFor(mapName, mkey, value)
storage.getMapStorageEntryFor(mapName, mkey).isDefined should be(true)
storage.removeMapStorageFor(mapName)
storage.getMapStorageEntryFor(mapName, mkey) should be(None)
}
it("should insert a single map storage element properly") {
val mapName = "insertSingleTest"
val mkey = "insertSingleTestKey".getBytes
val value = "insertSingleTestValue".getBytes
storage.insertMapStorageEntryFor(mapName, mkey, value)
storage.getMapStorageEntryFor(mapName, mkey).get should be(value)
storage.removeMapStorageFor(mapName, mkey)
storage.getMapStorageEntryFor(mapName, mkey) should be(None)
storage.insertMapStorageEntryFor(mapName, mkey, value)
storage.getMapStorageEntryFor(mapName, mkey).get should be(value)
storage.removeMapStorageFor(mapName)
storage.getMapStorageEntryFor(mapName, mkey) should be(None)
}
it("should insert multiple map storage elements properly") {
val mapName = "insertMultipleTest"
val rand = new Random(3).nextInt(100)
val entries = (1 to rand).toList.map{
index =>
(("insertMultipleTestKey" + index).getBytes -> ("insertMutlipleTestValue" + index).getBytes)
}
storage.insertMapStorageEntriesFor(mapName, entries)
entries foreach {
_ match {
case (mkey, value) => {
storage.getMapStorageEntryFor(mapName, mkey).isDefined should be(true)
storage.getMapStorageEntryFor(mapName, mkey).get should be(value)
}
}
}
storage.removeMapStorageFor(mapName)
entries foreach {
_ match {
case (mkey, value) => {
storage.getMapStorageEntryFor(mapName, mkey) should be(None)
}
}
}
}
it("should accurately track the number of key value pairs in a map") {
val mapName = "sizeTest"
val rand = new Random(3).nextInt(100)
val entries = (1 to rand).toList.map{
index =>
(("sizeTestKey" + index).getBytes -> ("sizeTestValue" + index).getBytes)
}
storage.insertMapStorageEntriesFor(mapName, entries)
storage.getMapStorageSizeFor(mapName) should be(rand)
}
it("should return all the key value pairs in the map in the correct order when getMapStorageFor(name) is called") {
val mapName = "allTest"
val rand = new Random(3).nextInt(100)
var entries = new TreeMap[Array[Byte], Array[Byte]]()(ArrayOrdering)
(1 to rand).foreach{
index =>
entries += (("allTestKey" + index).getBytes -> ("allTestValue" + index).getBytes)
}
storage.insertMapStorageEntriesFor(mapName, entries.toList)
val retrieved = storage.getMapStorageFor(mapName)
retrieved.size should be(rand)
entries.size should be(rand)
val entryMap = new HashMap[String, String] ++ entries.map{
_ match {
case (k, v) => (new String(k), new String(v))
}
}
val retrievedMap = new HashMap[String, String] ++ entries.map{
_ match {
case (k, v) => (new String(k), new String(v))
}
}
entryMap should equal(retrievedMap)
(0 until rand).foreach{
i: Int => {
new String(entries.toList(i)._1) should be(new String(retrieved(i)._1))
}
}
}
it("should return all the key->value pairs that exist in the map that are between start and end, up to count pairs when getMapStorageRangeFor is called") {
//implement if this method will be used
}
it("should return Some(null), not None, for a key that has had the value null set and None for a key with no value set") {
val mapName = "nullTest"
val key = "key".getBytes
storage.insertMapStorageEntryFor(mapName, key, null)
storage.getMapStorageEntryFor(mapName, key).get should be(null)
storage.removeMapStorageFor(mapName, key)
storage.getMapStorageEntryFor(mapName, key) should be(None)
}
it("should not throw an exception when size is called on a non existent map?") {
storage.getMapStorageSizeFor("nonExistent") should be(0)
}
it("should not stomp on the map keyset when a map key of 0xff is used") {
val mapName = "keySetStomp"
val key = CommonStorageBackend.mapKeysIndex
storage.insertMapStorageEntryFor(mapName, key, key)
storage.getMapStorageSizeFor(mapName) should be(1)
storage.getMapStorageEntryFor(mapName,key).get should be (key)
}
}
}

Some files were not shown because too many files have changed in this diff Show more