Merge branch '290-hseeberger'

This commit is contained in:
Heiko Seeberger 2010-07-02 22:49:32 +02:00
commit 79e32401bf
20 changed files with 40 additions and 40 deletions

View file

@ -86,7 +86,7 @@ trait CamelContextLifecycle extends Logging {
/**
* Initializes this lifecycle object with the a DefaultCamelContext.
*/
def init: Unit = init(new DefaultCamelContext)
def init(): Unit = init(new DefaultCamelContext)
/**
* Initializes this lifecycle object with the given CamelContext. For the passed

View file

@ -58,7 +58,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
assert(exchange.getOut.getHeader("k3") === null) // headers from failure message are currently ignored
}
@Test def shouldSendMessageToActorAndTimeout: Unit = {
@Test def shouldSendMessageToActorAndTimeout(): Unit = {
val actor = actorOf[Tester3].start
val endpoint = mockEndpoint("actor:uuid:%s" format actor.uuid)
val exchange = endpoint.createExchange(ExchangePattern.InOut)

View file

@ -386,7 +386,7 @@ trait ActorRef extends TransactionManagement {
* Invoking 'makeTransactionRequired' means that the actor will **start** a new transaction if non exists.
* However, it will always participate in an existing transaction.
*/
def makeTransactionRequired: Unit
def makeTransactionRequired(): Unit
/**
* Sets the transaction configuration for this actor. Needs to be invoked before the actor is started.
@ -429,12 +429,12 @@ trait ActorRef extends TransactionManagement {
* Shuts down the actor its dispatcher and message queue.
* Alias for 'stop'.
*/
def exit = stop
def exit() = stop()
/**
* Shuts down the actor its dispatcher and message queue.
*/
def stop: Unit
def stop(): Unit
/**
* Links an other actor to this actor. Links are unidirectional and means that a the linking actor will
@ -510,7 +510,7 @@ trait ActorRef extends TransactionManagement {
/**
* Shuts down and removes all linked actors.
*/
def shutdownLinkedActors: Unit
def shutdownLinkedActors(): Unit
protected[akka] def invoke(messageHandle: MessageInvocation): Unit
@ -680,7 +680,7 @@ sealed class LocalActorRef private[akka](
* Invoking 'makeTransactionRequired' means that the actor will **start** a new transaction if non exists.
* However, it will always participate in an existing transaction.
*/
def makeTransactionRequired = guard.withGuard {
def makeTransactionRequired() = guard.withGuard {
if (!isRunning || isBeingRestarted) isTransactor = true
else throw new ActorInitializationException(
"Can not make actor transaction required after it has been started")
@ -873,7 +873,7 @@ sealed class LocalActorRef private[akka](
/**
* Shuts down and removes all linked actors.
*/
def shutdownLinkedActors: Unit = guard.withGuard {
def shutdownLinkedActors(): Unit = guard.withGuard {
linkedActorsAsList.foreach(_.stop)
linkedActors.clear
}
@ -1221,7 +1221,7 @@ private[akka] case class RemoteActorRef private[akka] (
this
}
def stop: Unit = {
def stop(): Unit = {
_isRunning = false
_isShutDown = true
}
@ -1237,7 +1237,7 @@ private[akka] case class RemoteActorRef private[akka] (
def actorClass: Class[_ <: Actor] = unsupported
def dispatcher_=(md: MessageDispatcher): Unit = unsupported
def dispatcher: MessageDispatcher = unsupported
def makeTransactionRequired: Unit = unsupported
def makeTransactionRequired(): Unit = unsupported
def transactionConfig_=(config: TransactionConfig): Unit = unsupported
def transactionConfig: TransactionConfig = unsupported
def makeRemote(hostname: String, port: Int): Unit = unsupported
@ -1254,7 +1254,7 @@ private[akka] case class RemoteActorRef private[akka] (
def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef = unsupported
def mailboxSize: Int = unsupported
def supervisor: Option[ActorRef] = unsupported
def shutdownLinkedActors: Unit = unsupported
def shutdownLinkedActors(): Unit = unsupported
protected[akka] def mailbox: Deque[MessageInvocation] = unsupported
protected[akka] def restart(reason: Throwable): Unit = unsupported
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported

View file

@ -134,7 +134,7 @@ sealed class Supervisor private[akka] (
this
}
def shutdown: Unit = supervisor.stop
def shutdown(): Unit = supervisor.stop
def link(child: ActorRef) = supervisor.link(child)
@ -183,7 +183,7 @@ final class SupervisorActor private[akka] (
trapExit = trapExceptions
faultHandler = Some(handler)
override def shutdown: Unit = shutdownLinkedActors
override def shutdown(): Unit = shutdownLinkedActors
def receive = {
case unknown => throw new SupervisorException(

View file

@ -133,7 +133,7 @@ class ExecutorBasedEventDrivenDispatcher(_name: String, throughput: Int = Dispat
def usesActorMailbox = true
def ensureNotActive: Unit = if (active) throw new IllegalActorStateException(
def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
"Can't build a new thread pool for a dispatcher that is already up and running")
private[akka] def init = withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool

View file

@ -162,7 +162,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
references.clear
}
def ensureNotActive: Unit = if (active) throw new IllegalActorStateException(
def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
"Can't build a new thread pool for a dispatcher that is already up and running")
private[akka] def init = withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool

View file

@ -139,7 +139,7 @@ class ReactorBasedThreadPoolEventDrivenDispatcher(_name: String)
def usesActorMailbox = false
def ensureNotActive: Unit = if (active) throw new IllegalActorStateException(
def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
"Can't build a new thread pool for a dispatcher that is already up and running")
class Demultiplexer(private val messageQueue: ReactiveMessageQueue) extends MessageDemultiplexer {

View file

@ -31,7 +31,7 @@ trait ThreadPoolBuilder {
def isShutdown = executor.isShutdown
def buildThreadPool: Unit = synchronized {
def buildThreadPool(): Unit = synchronized {
ensureNotActive
inProcessOfBuilding = false
if (boundedExecutorBound > 0) {
@ -152,7 +152,7 @@ trait ThreadPoolBuilder {
"Is not in the process of building a thread pool, start building one by invoking one of the 'newThreadPool*' methods")
}
def ensureNotActive: Unit
def ensureNotActive(): Unit
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>

View file

@ -251,7 +251,7 @@ object Cluster extends Cluster with Logging {
def foreach(f: (RemoteAddress) => Unit): Unit = clusterActor.foreach(_.foreach(f))
def start: Unit = start(None)
def start(): Unit = start(None)
def start(serializerClassLoader: Option[ClassLoader]): Unit = synchronized {
log.info("Starting up Cluster Service...")
@ -274,7 +274,7 @@ object Cluster extends Cluster with Logging {
}
}
def shutdown: Unit = synchronized {
def shutdown(): Unit = synchronized {
log.info("Shutting down Cluster Service...")
for {
c <- clusterActorRef

View file

@ -38,10 +38,10 @@ class JGroupsClusterActor extends BasicClusterActor {
def suspect(a: Address): Unit =
if (isActive) self ! Zombie(a)
def block: Unit =
def block(): Unit =
log debug "UNSUPPORTED: JGroupsClusterActor::block" //TODO HotSwap to a buffering body
def unblock: Unit =
def unblock(): Unit =
log debug "UNSUPPORTED: JGroupsClusterActor::unblock" //TODO HotSwap back and flush the buffer
})
})
@ -52,7 +52,7 @@ class JGroupsClusterActor extends BasicClusterActor {
protected def toOneNode(dest : Address, msg: Array[Byte]): Unit =
for (c <- channel) c.send(new JG_MSG(dest, null, msg))
protected def toAllNodes(msg : Array[Byte]) : Unit =
protected def toAllNodes(msg : Array[Byte]): Unit =
for (c <- channel) c.send(new JG_MSG(null, null, msg))
override def shutdown = {

View file

@ -191,13 +191,13 @@ trait Transactional {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait Committable {
def commit: Unit
def commit(): Unit
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait Abortable {
def abort: Unit
def abort(): Unit
}

View file

@ -84,7 +84,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcherSpec extends JUnitSuite with
fast.stop
}
@Test def canNotUseActorsOfDifferentTypesInSameDispatcher: Unit = {
@Test def canNotUseActorsOfDifferentTypesInSameDispatcher(): Unit = {
val first = actorOf[FirstActor]
val second = actorOf[SecondActor]
@ -94,7 +94,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcherSpec extends JUnitSuite with
}
}
@Test def canNotUseActorsOfDifferentSubTypesInSameDispatcher: Unit = {
@Test def canNotUseActorsOfDifferentSubTypesInSameDispatcher(): Unit = {
val parent = actorOf[ParentActor]
val child = actorOf[ChildActor]

View file

@ -54,7 +54,7 @@ class ThreadBasedDispatcherSpec extends JUnitSuite {
internalTestMessagesDispatchedToHandlersAreExecutedInFIFOOrder
}
private def internalTestMessagesDispatchedToTheSameHandlerAreExecutedSequentially: Unit = {
private def internalTestMessagesDispatchedToTheSameHandlerAreExecutedSequentially(): Unit = {
val guardLock = new ReentrantLock
val handleLatch = new CountDownLatch(100)
val dispatcher = new ThreadBasedDispatcher("name", new TestMessageHandle(handleLatch))
@ -66,7 +66,7 @@ class ThreadBasedDispatcherSpec extends JUnitSuite {
assert(!threadingIssueDetected.get)
}
private def internalTestMessagesDispatchedToHandlersAreExecutedInFIFOOrder: Unit = {
private def internalTestMessagesDispatchedToHandlersAreExecutedInFIFOOrder(): Unit = {
val handleLatch = new CountDownLatch(100)
val dispatcher = new ThreadBasedDispatcher("name", new MessageInvoker {
var currentValue = -1;

View file

@ -27,7 +27,7 @@ class AkkaClusterBroadcastFilter extends Actor with ClusterBroadcastFilter {
/**
* Stops the actor
*/
def destroy: Unit = self.stop
def destroy(): Unit = self.stop
/**
* Relays all non ClusterCometBroadcast messages to the other AkkaClusterBroadcastFilters in the cluster

View file

@ -71,7 +71,7 @@ trait TransactionProtocol extends Logging {
protected val joinTransactionFuns: JList[() => Unit] = new CopyOnWriteArrayList[() => Unit]
protected val exceptionsNotToRollbackOn: JList[Class[_ <: Exception]] = new CopyOnWriteArrayList[Class[_ <: Exception]]
def joinTransaction: Unit = {
def joinTransaction(): Unit = {
val it = joinTransactionFuns.iterator
while (it.hasNext) {
val fn = it.next

View file

@ -44,7 +44,7 @@ trait EmbeddedAppServer extends Bootable with Logging {
val adapter = new ServletAdapter
adapter.setHandleStaticResources(true)
adapter.setServletInstance(new AkkaServlet {
override def init(sc : ServletConfig) : Unit = {
override def init(sc : ServletConfig): Unit = {
val cl = Thread.currentThread.getContextClassLoader
try {
Thread.currentThread.setContextClassLoader(applicationLoader.get)

View file

@ -23,13 +23,13 @@ object Kernel extends AkkaLoader {
/**
* Boots up the Kernel with default bootables
*/
def boot: Unit = boot(true,
def boot(): Unit = boot(true,
new EmbeddedAppServer with BootableActorLoaderService
with BootableRemoteActorService
with CamelService)
//For testing purposes only
def startRemoteService: Unit = bundles.foreach( _ match {
def startRemoteService(): Unit = bundles.foreach( _ match {
case x: BootableRemoteActorService => x.startRemoteService
case _ =>
})

View file

@ -168,7 +168,7 @@ object // EmbeddedCassandraService {
t.setDaemon(true)
t.start
def start: Unit = {}
def start(): Unit = {}
}
*/

View file

@ -13,10 +13,10 @@ trait Pool[T] extends java.io.Closeable {
def borrowObject: T
def returnObject(t: T): Unit
def invalidateObject(t: T): Unit
def addObject: Unit
def addObject(): Unit
def getNumIdle: Int
def getNumActive: Int
def clear: Unit
def clear(): Unit
def setFactory(factory: PoolItemFactory[T]): Unit
}
@ -40,8 +40,8 @@ trait PoolBridge[T, OP <: ObjectPool] extends Pool[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 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 {

View file

@ -183,7 +183,7 @@ trait ChatServer extends Actor {
// abstract methods to be defined somewhere else
protected def chatManagement: Receive
protected def sessionManagement: Receive
protected def shutdownSessions: Unit
protected def shutdownSessions(): Unit
override def shutdown = {
log.info("Chat server is shutting down...")