merged with master

This commit is contained in:
Jonas Bonér 2010-01-02 23:03:49 +01:00
commit 63387fb286
76 changed files with 384 additions and 570 deletions

View file

@ -17,4 +17,4 @@ the License.
---------------
Licenses for dependency projects can be found here:
[http://wiki.github.com/jboner/akka/licenses]
[http://doc.akkasource.org/licenses]

View file

@ -1,70 +1,15 @@
h1. Akka Transactors
h1. Akka
h2. Distributed Transactional RESTful Persistent Actors
We believe that writing correct concurrent, fault-tolerant and scalable applications is too hard. Most of the time it's because we are using the wrong tools and the wrong level of abstraction.
h3. "http://akkasource.org":http://akkasource.org
Akka is here to change that.
Akka implements a unique hybrid of:
* The Actor model (Actors and Active Objects), which gives you:
** Concurrency (high-level and simple)
** Asynchronous, non-blocking and highly performant components.
** Supervision with "let-it-crash" semantics. Components are loosely coupled and restarted upon failure.
* Software Transactional Memory (STM).
* BASE and ACID persistence - Pluggable Eventually Consistent or ACID distributed scalable persistent storage.
* Remoting - Distributed services with supervision and error management
* REST (JAX-RS) and Comet bindings.
* Monitoring and Management
Using the Actor Model together with Software Transactional Memory we raise the abstraction level and provides a better platform to build correct concurrent and scalable applications.
Akka can be used in two different ways:
* As a library: used by a web app, to be put into WEB-INF/lib
* As a kernel: stand-alone kernel, embedding the servlet container
For fault-tolerance we adopt the "Let it crash" / "Embrace failure" model which have been used with great success in the telecom industry to build applications that self-heals, systems that never stop.
See the "Use-case and Deployment Scenarios":http://wiki.github.com/jboner/akka/use-case-and-deployment-scenarios for details.
Actors also provides the abstraction for transparent distribution and the basis for truly scalable and fault-tolerant applications.
h1. What's Akka all about? Why should I care?
Akka is Open Source and available under the Apache 2 License.
If you are new to Akka then I suggest you start with either the:
* "High Level View":http://wiki.github.com/jboner/akka/modules-the-high-level-view; which is outlining the different modules in Akka.
* "Use-case and Deployment Scenarios":http://wiki.github.com/jboner/akka/use-case-and-deployment-scenarios; outlining how and in which use-case and deployment scenarios can I use Akka?
* "Examples":http://wiki.github.com/jboner/akka/examples; showing how to build a RESTful, transactional, persistent Active Object and Actor.
After that you can dive into the "Reference Manual":http://wiki.github.com/jboner/akka/akka-reference-manual.
h1. Documentation
Akka has pretty thorough "reference documentation":https://github.com/jboner/akka/wikis. Covering examples, APIs and configuration.
h1. Distribution
The latest distribution can be found in the "downloads section":https://github.com/jboner/akka/downloads
h1. Mailing List
If you have questions and/or feedback: please sign up to the Akka User mailing list:
"http://groups.google.com/group/akka-user":http://groups.google.com/group/akka-user
h1. Professional Support
Scalable Solutions AB is providing a variety of professional support packages for Akka, please visit their website for details:
"http://scalablesolutions.se":http://scalablesolutions.se
h1. License
<pre>
This software is licensed under the Apache 2 license, quoted below.
Copyright 2009 Scalable Solutions AB <http://scalablesolutions.se>
Licensed 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.
</pre>
Learn more at "http://akkasource.org":http://akkasource.org

View file

@ -11,15 +11,9 @@
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>
<dependency>
<artifactId>akka-util</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>

View file

@ -12,16 +12,10 @@
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<!-- Core deps -->
<dependencies>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-rest</artifactId>
<groupId>${project.groupId}</groupId>
@ -34,11 +28,7 @@
<artifactId>grizzly-comet-webserver</artifactId>
<version>${grizzly.version}</version>
</dependency>
<!--dependency>
<groupId>com.sun.grizzly</groupId>
<artifactId>grizzly-comet-webserver</artifactId>
<version>${grizzly.version}</version>
</dependency-->
<!-- For Atmosphere -->
<dependency>
<groupId>javax.servlet</groupId>
@ -60,30 +50,5 @@
<artifactId>atmosphere-runtime</artifactId>
<version>${atmosphere.version}</version>
</dependency>
<!--dependency>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-core</artifactId>
<version>${jersey.version}</version>
</dependency>
<dependency>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-server</artifactId>
<version>${jersey.version}</version>
</dependency-->
<!--dependency>
<groupId>com.sun.jersey</groupId>
<artifactId>jersey-json</artifactId>
<version>${jersey.version}</version>
</dependency>
<dependency>
<groupId>javax.ws.rs</groupId>
<artifactId>jsr311-api</artifactId>
<version>1.1</version>
</dependency>
<dependency>
<groupId>com.sun.jersey.contribs</groupId>
<artifactId>jersey-scala</artifactId>
<version>${jersey.version}</version>
</dependency-->
</dependencies>
</project>

View file

@ -12,7 +12,6 @@
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<!-- Core deps -->

View file

@ -415,6 +415,10 @@ trait Actor extends TransactionManagement {
def start: Actor = synchronized {
if (_isShutDown) throw new IllegalStateException("Can't restart an actor that has been shut down with 'exit'")
if (!_isRunning) {
if (messageDispatcher.isShutdown &&
messageDispatcher.isInstanceOf[Dispatchers.globalExecutorBasedEventDrivenDispatcher.type]) {
messageDispatcher.asInstanceOf[ExecutorBasedEventDrivenDispatcher].init
}
messageDispatcher.register(this)
messageDispatcher.start
_isRunning = true

View file

@ -57,9 +57,8 @@ class ExecutorBasedEventDrivenDispatcher(_name: String) extends MessageDispatche
@volatile private var active: Boolean = false
val name: String = "event-driven:executor:dispatcher:" + _name
withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool
init
def dispatch(invocation: MessageInvocation) = if (active) {
executor.execute(new Runnable() {
def run = {
@ -79,10 +78,14 @@ class ExecutorBasedEventDrivenDispatcher(_name: String) extends MessageDispatche
}
def shutdown = if (active) {
log.debug("Shutting down ThreadBasedDispatcher [%s]", name)
executor.shutdownNow
active = false
references.clear
}
def ensureNotActive: Unit = if (active) throw new IllegalStateException(
"Can't build a new thread pool for a dispatcher that is already up and running")
private[akka] def init = withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool
}

View file

@ -6,7 +6,7 @@ package se.scalablesolutions.akka.dispatch
import java.util.List
import se.scalablesolutions.akka.util.HashCode
import se.scalablesolutions.akka.util.{HashCode, Logging}
import se.scalablesolutions.akka.stm.Transaction
import se.scalablesolutions.akka.actor.Actor
@ -56,7 +56,7 @@ trait MessageInvoker {
def invoke(message: MessageInvocation)
}
trait MessageDispatcher {
trait MessageDispatcher extends Logging {
protected val references = new ConcurrentHashMap[String, Actor]
def dispatch(invocation: MessageInvocation)
def start
@ -64,6 +64,7 @@ trait MessageDispatcher {
def register(actor: Actor) = references.put(actor.uuid, actor)
def unregister(actor: Actor) = references.remove(actor.uuid)
def canBeShutDown: Boolean = references.isEmpty
def isShutdown: Boolean
}
trait MessageDemultiplexer {

View file

@ -36,6 +36,8 @@ class ReactorBasedSingleThreadEventDrivenDispatcher(name: String) extends Abstra
selectorThread.start
}
def isShutdown = !active
class Demultiplexer(private val messageQueue: ReactiveMessageQueue) extends MessageDemultiplexer {
private val selectedQueue: List[MessageInvocation] = new LinkedList[MessageInvocation]

View file

@ -39,9 +39,13 @@ class ThreadBasedDispatcher private[akka] (val name: String, val messageHandler:
selectorThread.start
}
def isShutdown = !active
def shutdown = if (active) {
log.debug("Shutting down ExecutorBasedEventDrivenDispatcher [%s]", name)
active = false
selectorThread.interrupt
references.clear
}
}

View file

@ -28,6 +28,8 @@ trait ThreadPoolBuilder {
protected var executor: ExecutorService = _
def isShutdown = executor.isShutdown
def buildThreadPool = synchronized {
ensureNotActive
inProcessOfBuilding = false

View file

@ -32,11 +32,12 @@ trait BootableRemoteActorService extends Bootable with Logging {
}
abstract override def onUnload = {
super.onUnload
super.onUnload
if (remoteServerThread.isAlive) {
log.info("Shutting down Remote Actors Service")
RemoteNode.shutdown
log.info("Shutting down Cluster Service")
Cluster.shutdown
remoteServerThread.join(1000)
}
}

View file

@ -8,11 +8,11 @@ import org.jgroups.{JChannel, View, Address, Message, ExtendedMembershipListener
import se.scalablesolutions.akka.Config.config
import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor, ActorRegistry}
import se.scalablesolutions.akka.remote.Cluster.{Node, RelayedMessage}
import se.scalablesolutions.akka.serialization.Serializer
import se.scalablesolutions.akka.actor.{Supervisor, SupervisorFactory, Actor, ActorRegistry}
import scala.collection.immutable.{Map, HashMap}
import se.scalablesolutions.akka.serialization.Serializer
/**
* Interface for interacting with the Cluster Membership API.
@ -48,20 +48,24 @@ object Cluster extends Cluster {
private[remote] case class Node(endpoints: List[RemoteAddress]) extends ClusterMessage
private[remote] case class RelayedMessage(actorClassFQN: String, msg: AnyRef) extends ClusterMessage
private[remote] lazy val clusterActor: Option[ClusterActor] = {
config.getString("akka.remote.cluster.actor") map (name => {
val actor = Class.forName(name)
.newInstance
.asInstanceOf[ClusterActor]
SupervisorFactory(
SupervisorConfig(
RestartStrategy(OneForOne, 5, 1000, List(classOf[Exception])),
Supervise(actor, LifeCycle(Permanent)) :: Nil)
).newInstance.start
actor
})
}
private[remote] val clusterActor: Option[ClusterActor] =
config.getString("akka.remote.cluster.actor") map { name =>
val a = Class.forName(name).newInstance.asInstanceOf[ClusterActor]
a.start
a
}
private[remote] val supervisor: Option[Supervisor] = if (clusterActor.isDefined) {
val sup = SupervisorFactory(
SupervisorConfig(
RestartStrategy(OneForOne, 5, 1000, List(classOf[Exception])),
Supervise(clusterActor.get, LifeCycle(Permanent)) :: Nil)
).newInstance
sup.start
Some(sup)
} else None
private[remote] lazy val serializer: Serializer = {
val className = config.getString("akka.remote.cluster.serializer", Serializer.Java.getClass.getName)
Class.forName(className).newInstance.asInstanceOf[Serializer]
@ -71,11 +75,13 @@ object Cluster extends Cluster {
def lookup[T](pf: PartialFunction[RemoteAddress, T]): Option[T] = clusterActor.flatMap(_.lookup(pf))
def registerLocalNode(hostname: String, port: Int): Unit = clusterActor.map(_.registerLocalNode(hostname, port))
def registerLocalNode(hostname: String, port: Int): Unit = clusterActor.foreach(_.registerLocalNode(hostname, port))
def deregisterLocalNode(hostname: String, port: Int): Unit = clusterActor.map(_.deregisterLocalNode(hostname, port))
def deregisterLocalNode(hostname: String, port: Int): Unit = clusterActor.foreach(_.deregisterLocalNode(hostname, port))
def relayMessage(to: Class[_ <: Actor], msg: AnyRef): Unit = clusterActor.map(_.relayMessage(to, msg))
def relayMessage(to: Class[_ <: Actor], msg: AnyRef): Unit = clusterActor.foreach(_.relayMessage(to, msg))
def shutdown = supervisor.foreach(_.stop)
}
/**
@ -99,6 +105,8 @@ class JGroupsClusterActor extends ClusterActor {
import JGroupsClusterActor._
import org.scala_tools.javautils.Implicits._
@volatile private var isActive = false
@volatile private var local: Node = Node(Nil)
@volatile private var channel: Option[JChannel] = None
@volatile private var remotes: Map[Address, Node] = Map()
@ -107,6 +115,7 @@ class JGroupsClusterActor extends ClusterActor {
log debug "Initiating JGroups-based cluster actor"
remotes = new HashMap[Address, Node]
val me = this
isActive = true
// Set up the JGroups local endpoint
channel = Some(new JChannel {
@ -115,15 +124,15 @@ class JGroupsClusterActor extends ClusterActor {
def setState(state: Array[Byte]): Unit = ()
def receive(msg: Message): Unit = me send msg
def receive(msg: Message): Unit = if (isActive) me send msg
def viewAccepted(view: View): Unit = me send view
def viewAccepted(view: View): Unit = if (isActive) me send view
def suspect(a: Address): Unit = me send Zombie(a)
def suspect(a: Address): Unit = if (isActive) me send Zombie(a)
def block: Unit = me send Block
def block: Unit = if (isActive) me send Block
def unblock: Unit = me send Unblock
def unblock: Unit = if (isActive) me send Unblock
})
})
channel.map(_.connect(name))
@ -213,8 +222,9 @@ class JGroupsClusterActor extends ClusterActor {
}
override def shutdown = {
log debug ("Shutting down %s", this.getClass.getName)
channel.map(_.shutdown)
log debug ("Shutting down %s", toString)
isActive = false
channel.foreach(_.shutdown)
remotes = Map()
channel = None
}

View file

@ -43,7 +43,7 @@ import org.jboss.netty.handler.codec.compression.{ZlibEncoder, ZlibDecoder}
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object RemoteNode extends RemoteServer
object RemoteNode extends RemoteServer(true)
/**
* This object holds configuration variables.
@ -116,8 +116,9 @@ object RemoteServer {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteServer extends Logging {
class RemoteServer(val registerNodeInCluster: Boolean) extends Logging {
val name = "RemoteServer@" + hostname + ":" + port
def this() = this(false)
private var hostname = RemoteServer.HOSTNAME
private var port = RemoteServer.PORT
@ -155,7 +156,7 @@ class RemoteServer extends Logging {
bootstrap.setOption("child.connectTimeoutMillis", RemoteServer.CONNECTION_TIMEOUT_MILLIS)
openChannels.add(bootstrap.bind(new InetSocketAddress(hostname, port)))
isRunning = true
Cluster.registerLocalNode(hostname, port)
if (registerNodeInCluster) Cluster.registerLocalNode(hostname, port)
}
} catch {
case e => log.error(e, "Could not start up remote server")
@ -163,9 +164,14 @@ class RemoteServer extends Logging {
}
def shutdown = {
openChannels.close.awaitUninterruptibly()
openChannels.disconnect
openChannels.unbind
openChannels.close.awaitUninterruptibly(1000)
bootstrap.releaseExternalResources
Cluster.deregisterLocalNode(hostname, port)
if (registerNodeInCluster) {
Cluster.deregisterLocalNode(hostname, port)
Cluster.shutdown
}
}
}

View file

@ -0,0 +1,34 @@
package test
import se.scalablesolutions.akka.actor.Actor
import se.scalablesolutions.akka.remote.RemoteNode
object AkkaTest1 {
def main(args: Array[String]) {
class MyActor extends Actor {
def receive = {
case "test" => println("received test")
case m@_ => println("received unknown message " + m)
}
}
val myActor = new MyActor
myActor.start
myActor.send("test")
myActor.stop
// does not exit
}
}
// case 2
object AkkaTest2 {
def main(args: Array[String]) {
RemoteNode.start("localhost", 9999)
Thread.sleep(3000)
RemoteNode.shutdown
// does not exit
}
}

View file

@ -8,7 +8,7 @@ public class AllTest extends TestCase {
public static Test suite() {
TestSuite suite = new TestSuite("All Java tests");
suite.addTestSuite(InMemoryStateTest.class);
suite.addTestSuite(InMemNestedStateTest.class);
//suite.addTestSuite(InMemNestedStateTest.class);
suite.addTestSuite(RemoteInMemoryStateTest.class);
suite.addTestSuite(ActiveObjectGuiceConfiguratorTest.class);
//suite.addTestSuite(PersistentStateTest.class);

View file

@ -11,7 +11,7 @@ import static se.scalablesolutions.akka.config.JavaConfig.*;
import se.scalablesolutions.akka.actor.*;
import se.scalablesolutions.akka.Kernel;
import junit.framework.TestCase;
/*
public class InMemNestedStateTest extends TestCase {
static String messageLog = "";
@ -133,3 +133,4 @@ public class InMemNestedStateTest extends TestCase {
assertEquals("init", nested.getRefState()); // check that state is == init state
}
}
*/

View file

@ -93,7 +93,7 @@ public class InMemoryStateTest extends TestCase {
} // expected
assertEquals("init", stateful.getRefState()); // check that state is == init state
}
/*
/*
public void testNestedNonTransactionalMethodHangs() {
InMemStateful stateful = conf.getInstance(InMemStateful.class);
stateful.setMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure", "init"); // set init state

View file

@ -12,16 +12,10 @@
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<!-- akka deps -->
<dependencies>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-rest</artifactId>
<groupId>${project.groupId}</groupId>

View file

@ -8,10 +8,9 @@
<packaging>jar</packaging>
<parent>
<artifactId>akka</artifactId>
<artifactId>akka-persistence-parent</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>
@ -32,20 +31,6 @@
<artifactId>log4j</artifactId>
<version>1.2.13</version>
</dependency>
<!-- For Testing -->
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest</artifactId>
<version>1.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>4.5</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View file

@ -8,19 +8,12 @@
<packaging>jar</packaging>
<parent>
<artifactId>akka</artifactId>
<artifactId>akka-persistence-parent</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>com.facebook</groupId>
<artifactId>thrift</artifactId>
@ -31,20 +24,6 @@
<artifactId>commons-pool</artifactId>
<version>1.5.1</version>
</dependency>
<!-- For Testing -->
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest</artifactId>
<version>1.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>4.5</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View file

@ -8,10 +8,9 @@
<packaging>jar</packaging>
<parent>
<artifactId>akka</artifactId>
<artifactId>akka-persistence-parent</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>
@ -27,20 +26,6 @@
<artifactId>mongo-java-driver</artifactId>
<version>1.1</version>
</dependency>
<!-- For Testing -->
<dependency>
<groupId>org.scalatest</groupId>
<artifactId>scalatest</artifactId>
<version>1.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>4.5</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View file

@ -0,0 +1,31 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-persistence-redis</artifactId>
<name>Akka Persistence Redis Module</name>
<packaging>jar</packaging>
<parent>
<artifactId>akka-persistence-parent</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
</parent>
<dependencies>
<dependency>
<artifactId>akka-persistence-common</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<!-- For Redis -->
<dependency>
<groupId>com.redis</groupId>
<artifactId>redisclient</artifactId>
<version>1.0.1</version>
</dependency>
</dependencies>
</project>

View file

@ -2,32 +2,30 @@
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-persistence-redis</artifactId>
<name>Akka Persistence Redis Module</name>
<artifactId>akka-persistence-parent</artifactId>
<name>Akka Persistence Modules</name>
<packaging>jar</packaging>
<packaging>pom</packaging>
<parent>
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<modules>
<module>akka-persistence-common</module>
<module>akka-persistence-redis</module>
<module>akka-persistence-mongo</module>
<module>akka-persistence-cassandra</module>
</modules>
<dependencies>
<dependency>
<artifactId>akka-persistence-common</artifactId>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<!-- For Mongo -->
<dependency>
<groupId>com.redis</groupId>
<artifactId>redisclient</artifactId>
<version>1.0.1</version>
</dependency>
<!-- For Testing -->
<dependency>
<groupId>org.scalatest</groupId>
@ -42,5 +40,4 @@
<scope>test</scope>
</dependency>
</dependencies>
</project>

View file

@ -12,7 +12,6 @@
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<!-- Core deps -->

View file

@ -1,88 +0,0 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-samples-java</artifactId>
<name>Akka Java Samples Module</name>
<packaging>jar</packaging>
<parent>
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>
<dependency>
<artifactId>akka-util-java</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-util</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-rest</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>akka-persistence-cassandra</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-kernel</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>javax.ws.rs</groupId>
<artifactId>jsr311-api</artifactId>
<version>1.0</version>
</dependency>
</dependencies>
<build>
<sourceDirectory>src/main/java</sourceDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.5</source>
<target>1.5</target>
<includes>
<include>**/*</include>
</includes>
</configuration>
</plugin>
<plugin>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<phase>install</phase>
<configuration>
<tasks>
<copy file="target/akka-samples-java-${project.version}.jar"
tofile="../deploy/akka-samples-java-${project.version}.jar"/>
</tasks>
</configuration>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View file

@ -1,97 +0,0 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-samples-lift</artifactId>
<name>Akka Lift Samples Module</name>
<packaging>war</packaging>
<parent>
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<properties>
<lift.version>1.1-M6</lift.version>
</properties>
<dependencies>
<dependency>
<artifactId>akka-util-java</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-util</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-persistence-cassandra</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-rest</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-kernel</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>javax.ws.rs</groupId>
<artifactId>jsr311-api</artifactId>
<version>1.0</version>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
</dependency>
<dependency>
<groupId>net.liftweb</groupId>
<artifactId>lift-util</artifactId>
<version>${lift.version}</version>
</dependency>
<dependency>
<groupId>net.liftweb</groupId>
<artifactId>lift-webkit</artifactId>
<version>${lift.version}</version>
</dependency>
<dependency>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
<version>2.5</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>4.5</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty</artifactId>
<version>[6.1.6,)</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-compiler</artifactId>
<version>${scala.version}</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View file

@ -1,78 +0,0 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-samples-security</artifactId>
<name>Akka Sample Security Module</name>
<packaging>jar</packaging>
<parent>
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>
<dependency>
<artifactId>akka-util-java</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-util</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-security</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-persistence-cassandra</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>javax.ws.rs</groupId>
<artifactId>jsr311-api</artifactId>
<version>1.0</version>
</dependency>
<dependency>
<groupId>javax.annotation</groupId>
<artifactId>jsr250-api</artifactId>
<version>1.0</version>
</dependency>
</dependencies>
<build>
<sourceDirectory>src/main/scala</sourceDirectory>
<plugins>
<plugin>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<phase>install</phase>
<configuration>
<tasks>
<copy file="target/akka-samples-security-${project.version}.jar"
tofile="../deploy/akka-samples-security-${project.version}.jar"/>
</tasks>
</configuration>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View file

@ -0,0 +1,38 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-sample-chat</artifactId>
<name>Akka Chat Sample Module</name>
<packaging>jar</packaging>
<parent>
<artifactId>akka-samples-parent</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
</parent>
<build>
<sourceDirectory>src/main/scala</sourceDirectory>
<plugins>
<plugin>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<phase>install</phase>
<configuration>
<tasks>
<copy file="target/akka-sample-chat-${project.version}.jar"
tofile="../../deploy/akka-sample-chat-${project.version}.jar"/>
</tasks>
</configuration>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View file

@ -0,0 +1,50 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-sample-lift</artifactId>
<name>Akka Lift Sample Module</name>
<packaging>war</packaging>
<parent>
<artifactId>akka-samples-parent</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
</parent>
<properties>
<lift.version>1.1-M6</lift.version>
</properties>
<dependencies>
<dependency>
<groupId>net.liftweb</groupId>
<artifactId>lift-util</artifactId>
<version>${lift.version}</version>
</dependency>
<dependency>
<groupId>net.liftweb</groupId>
<artifactId>lift-webkit</artifactId>
<version>${lift.version}</version>
</dependency>
<dependency>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
<version>2.5</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<version>4.5</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mortbay.jetty</groupId>
<artifactId>jetty</artifactId>
<version>[6.1.6,)</version>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View file

@ -0,0 +1,49 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-sample-rest-java</artifactId>
<name>Akka REST Java Sample Module</name>
<packaging>jar</packaging>
<parent>
<artifactId>akka-samples-parent</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
</parent>
<build>
<sourceDirectory>src/main/java</sourceDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.5</source>
<target>1.5</target>
<includes>
<include>**/*</include>
</includes>
</configuration>
</plugin>
<plugin>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<phase>install</phase>
<configuration>
<tasks>
<copy file="target/akka-sample-rest-java-${project.version}.jar"
tofile="../../deploy/akka-sample-rest-java-${project.version}.jar"/>
</tasks>
</configuration>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View file

@ -0,0 +1,46 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-sample-rest-scala</artifactId>
<name>Akka REST Scala Sample Module</name>
<packaging>jar</packaging>
<parent>
<artifactId>akka-samples-parent</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
</parent>
<dependencies>
<dependency>
<groupId>javax.ws.rs</groupId>
<artifactId>jsr311-api</artifactId>
<version>1.0</version>
</dependency>
</dependencies>
<build>
<sourceDirectory>src/main/scala</sourceDirectory>
<plugins>
<plugin>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<phase>install</phase>
<configuration>
<tasks>
<copy file="target/akka-sample-rest-scala-${project.version}.jar"
tofile="../../deploy/akka-sample-rest-scala-${project.version}.jar"/>
</tasks>
</configuration>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View file

@ -2,29 +2,29 @@
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-samples-chat</artifactId>
<name>Akka Chat Sample Module</name>
<artifactId>akka-sample-security</artifactId>
<name>Akka Sample Security Module</name>
<packaging>jar</packaging>
<parent>
<artifactId>akka</artifactId>
<artifactId>akka-samples-parent</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
<groupId>javax.ws.rs</groupId>
<artifactId>jsr311-api</artifactId>
<version>1.0</version>
</dependency>
<dependency>
<artifactId>akka-persistence-redis</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
<groupId>javax.annotation</groupId>
<artifactId>jsr250-api</artifactId>
<version>1.0</version>
</dependency>
</dependencies>
<build>
@ -37,8 +37,8 @@
<phase>install</phase>
<configuration>
<tasks>
<copy file="target/akka-samples-chat-${project.version}.jar"
tofile="../deploy/akka-samples-chat-${project.version}.jar"/>
<copy file="target/akka-sample-security-${project.version}.jar"
tofile="../../deploy/akka-sample-security-${project.version}.jar"/>
</tasks>
</configuration>
<goals>

View file

@ -2,29 +2,26 @@
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>akka-samples-scala</artifactId>
<name>Akka Scala Samples Module</name>
<artifactId>akka-samples-parent</artifactId>
<name>Akka Sample Modules</name>
<packaging>jar</packaging>
<packaging>pom</packaging>
<parent>
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<modules>
<module>akka-sample-chat</module>
<module>akka-sample-lift</module>
<module>akka-sample-security</module>
<module>akka-sample-rest-scala</module>
<module>akka-sample-rest-java</module>
</modules>
<dependencies>
<dependency>
<artifactId>akka-util-java</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-util</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
@ -35,43 +32,25 @@
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-persistence-redis</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-rest</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-kernel</artifactId>
<artifactId>akka-comet</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>javax.ws.rs</groupId>
<artifactId>jsr311-api</artifactId>
<version>1.0</version>
<artifactId>akka-security</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
</dependencies>
<build>
<sourceDirectory>src/main/scala</sourceDirectory>
<plugins>
<plugin>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<phase>install</phase>
<configuration>
<tasks>
<copy file="target/akka-samples-scala-${project.version}.jar"
tofile="../deploy/akka-samples-scala-${project.version}.jar"/>
</tasks>
</configuration>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View file

@ -11,30 +11,14 @@
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>${scala.version}</version>
</dependency>
<dependency>
<artifactId>akka-core</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-persistence-cassandra</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<artifactId>akka-util</artifactId>
<groupId>${project.groupId}</groupId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>javax.annotation</groupId>
<artifactId>jsr250-api</artifactId>

View file

@ -11,7 +11,6 @@
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>

View file

@ -11,7 +11,6 @@
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
<relativePath>../pom.xml</relativePath>
</parent>
<dependencies>

15
pom.xml
View file

@ -4,7 +4,7 @@
<modelVersion>4.0.0</modelVersion>
<name>Akka Transactors</name>
<name>Akka Project</name>
<artifactId>akka</artifactId>
<groupId>se.scalablesolutions.akka</groupId>
<version>0.6</version>
@ -51,22 +51,15 @@
<module>akka-util-java</module>
<module>akka-util</module>
<module>akka-core</module>
<module>akka-persistence-common</module>
<module>akka-persistence-cassandra</module>
<module>akka-persistence-mongo</module>
<module>akka-persistence-redis</module>
<module>akka-persistence</module>
<module>akka-rest</module>
<module>akka-comet</module>
<module>akka-amqp</module>
<module>akka-security</module>
<module>akka-spring</module>
<module>akka-kernel</module>
<module>akka-fun-test-java</module>
<module>akka-samples-scala</module>
<module>akka-samples-lift</module>
<module>akka-samples-chat</module>
<module>akka-samples-java</module>
<module>akka-samples-security</module>
<module>akka-spring</module>
<module>akka-samples</module>
</modules>
<organization>

View file

@ -1,4 +1,3 @@
#!/bin/sh
echo "removing all trailing whitespace from all *.scala, *.html and *.xml files"
find . -type f -name '*.scala' -exec sed -i 's/[ \t]*$//' {} \;
find . -type f -name '*.scala' -exec sed -i 's/[ \t]*$//' {} \;