Remove procedure syntax (#25362)

This commit is contained in:
kenji yoshida 2018-07-25 20:38:27 +09:00 committed by Johan Andrén
parent 50979d599c
commit 5b3b191bac
180 changed files with 403 additions and 403 deletions

View file

@ -25,7 +25,7 @@ trait Consumer extends Actor with CamelSupport {
* Registers the consumer endpoint. Note: when overriding this method, be sure to
* call 'super.preRestart', otherwise the consumer endpoint will not be registered.
*/
override def preStart() {
override def preStart(): Unit = {
super.preStart()
// Possible FIXME. registering the endpoint here because of problems
// with order of execution of trait body in the Java version (UntypedConsumerActor)
@ -34,7 +34,7 @@ trait Consumer extends Actor with CamelSupport {
register()
}
private[this] def register() {
private[this] def register(): Unit = {
camel.supervisor ! Register(self, endpointUri, Some(ConsumerConfig(activationTimeout, replyTimeout, autoAck, onRouteDefinition)))
}

View file

@ -18,12 +18,12 @@ trait ProducerSupport extends Actor with CamelSupport {
private[this] var messages = Vector.empty[(ActorRef, Any)]
private[this] var producerChild: Option[ActorRef] = None
override def preStart() {
override def preStart(): Unit = {
super.preStart()
register()
}
private[this] def register() { camel.supervisor ! Register(self, endpointUri) }
private[this] def register(): Unit = { camel.supervisor ! Register(self, endpointUri) }
/**
* CamelMessage headers to copy by default from request message to response-message.

View file

@ -146,9 +146,9 @@ private[camel] class Registry(activationTracker: ActorRef) extends Actor with Ca
}
}
private def deRegisterConsumer(actorRef: ActorRef) { consumerRegistrar ! DeRegister(actorRef) }
private def deRegisterConsumer(actorRef: ActorRef): Unit = { consumerRegistrar ! DeRegister(actorRef) }
private def deRegisterProducer(actorRef: ActorRef) { producerRegistrar ! DeRegister(actorRef) }
private def deRegisterProducer(actorRef: ActorRef): Unit = { producerRegistrar ! DeRegister(actorRef) }
}
/**

View file

@ -126,7 +126,7 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex
*/
private[camel] def processExchangeAdapter(exchange: CamelExchangeAdapter): Unit = {
val isDone = new CountDownLatch(1)
processExchangeAdapter(exchange, new AsyncCallback { def done(doneSync: Boolean) { isDone.countDown() } })
processExchangeAdapter(exchange, new AsyncCallback { def done(doneSync: Boolean): Unit = { isDone.countDown() } })
isDone.await(endpoint.replyTimeout.length, endpoint.replyTimeout.unit)
}

View file

@ -37,7 +37,7 @@ class ActivationIntegrationTest extends WordSpec with Matchers with SharedCamelS
def endpointUri = "direct:a3"
def receive = { case _ {} }
override def postStop() {
override def postStop(): Unit = {
super.postStop()
latch.countDown()
}
@ -69,7 +69,7 @@ class ActivationIntegrationTest extends WordSpec with Matchers with SharedCamelS
class TestConsumer(uri: String, latch: TestLatch) extends Consumer {
def endpointUri = uri
override def preStart() {
override def preStart(): Unit = {
Await.ready(latch, 60 seconds)
super.preStart()
}

View file

@ -62,7 +62,7 @@ class ConcurrentActivationTest extends WordSpec with Matchers with NonSharedCame
// should be the size of the activated activated producers and consumers
deactivations.size should ===(2 * number * number)
def partitionNames(refs: immutable.Seq[ActorRef]) = refs.map(_.path.name).partition(_.startsWith("concurrent-test-echo-consumer"))
def assertContainsSameElements(lists: (Seq[_], Seq[_])) {
def assertContainsSameElements(lists: (Seq[_], Seq[_])): Unit = {
val (a, b) = lists
a.intersect(b).size should ===(a.size)
}
@ -144,7 +144,7 @@ class Registrar(val start: Int, val number: Int, activationsPromise: Promise[Lis
}
}
def add(actor: Actor, name: String) {
def add(actor: Actor, name: String): Unit = {
val ref = context.actorOf(Props(actor), name)
actorRefs = actorRefs + ref
val result = camel.activationFutureFor(ref)

View file

@ -72,7 +72,7 @@ class ConsumerIntegrationTest extends WordSpec with Matchers with NonSharedCamel
case m: CamelMessage sender() ! "received " + m.bodyAs[String]
}
override def postRestart(reason: Throwable) {
override def postRestart(reason: Throwable): Unit = {
restarted.countDown()
}
}, "direct-a2")
@ -182,7 +182,7 @@ class ErrorThrowingConsumer(override val endpointUri: String) extends Consumer {
def receive = {
case msg: CamelMessage throw new TestException("error: %s" format msg.body)
}
override def preRestart(reason: Throwable, message: Option[Any]) {
override def preRestart(reason: Throwable, message: Option[Any]): Unit = {
super.preRestart(reason, message)
sender() ! Failure(reason)
}
@ -197,7 +197,7 @@ class ErrorRespondingConsumer(override val endpointUri: String) extends Consumer
rd.onException(classOf[TestException]).handled(true).transform(Builder.body.append(" has an error")).end
}
final override def preRestart(reason: Throwable, message: Option[Any]) {
final override def preRestart(reason: Throwable, message: Option[Any]): Unit = {
super.preRestart(reason, message)
sender() ! Failure(reason)
}
@ -213,7 +213,7 @@ class FailingOnceConsumer(override val endpointUri: String) extends Consumer {
throw new TestException("rejected: %s" format msg.body)
}
final override def preRestart(reason: Throwable, message: Option[Any]) {
final override def preRestart(reason: Throwable, message: Option[Any]): Unit = {
super.preRestart(reason, message)
sender() ! Failure(reason)
}

View file

@ -27,7 +27,7 @@ class ProducerFeatureTest extends TestKit(ActorSystem("ProducerFeatureTest", Akk
import ProducerFeatureTest._
implicit def camel = CamelExtension(system)
override protected def afterAll() {
override protected def afterAll(): Unit = {
super.afterAll()
shutdown()
}
@ -37,9 +37,9 @@ class ProducerFeatureTest extends TestKit(ActorSystem("ProducerFeatureTest", Akk
camelContext.setUseBreadcrumb(false)
val timeoutDuration = 1 second
implicit val timeout = Timeout(timeoutDuration)
override protected def beforeAll { camelContext.addRoutes(new TestRoute(system)) }
override protected def beforeAll: Unit = { camelContext.addRoutes(new TestRoute(system)) }
override protected def afterEach { mockEndpoint.reset() }
override protected def afterEach: Unit = { mockEndpoint.reset() }
"A Producer on a sync Camel route" must {
@ -279,7 +279,7 @@ object ProducerFeatureTest {
else msg
}
override def postStop() {
override def postStop(): Unit = {
for (msg lastMessage; aref lastSender) context.parent ! ((aref, msg))
super.postStop()
}
@ -288,7 +288,7 @@ object ProducerFeatureTest {
class TestProducer(uri: String, upper: Boolean = false) extends Actor with Producer {
def endpointUri = uri
override def preRestart(reason: Throwable, message: Option[Any]) {
override def preRestart(reason: Throwable, message: Option[Any]): Unit = {
//overriding on purpose so it doesn't try to deRegister and reRegister at restart,
// which would cause a deadletter message in the test output.
}
@ -339,7 +339,7 @@ object ProducerFeatureTest {
class TestRoute(system: ActorSystem) extends RouteBuilder {
val responder = system.actorOf(Props[TestResponder], name = "TestResponder")
def configure {
def configure: Unit = {
from("direct:forward-test-1").to("mock:mock")
// for one-way messaging tests
from("direct:producer-test-1").to("mock:mock")

View file

@ -21,7 +21,7 @@ private[camel] object TestSupport {
def start(actor: Actor, name: String)(implicit system: ActorSystem, timeout: Timeout): ActorRef =
Await.result(CamelExtension(system).activationFutureFor(system.actorOf(Props(actor), name))(timeout, system.dispatcher), timeout.duration)
def stop(actorRef: ActorRef)(implicit system: ActorSystem, timeout: Timeout) {
def stop(actorRef: ActorRef)(implicit system: ActorSystem, timeout: Timeout): Unit = {
system.stop(actorRef)
Await.result(CamelExtension(system).deactivationFutureFor(actorRef)(timeout, system.dispatcher), timeout.duration)
}
@ -51,7 +51,7 @@ private[camel] object TestSupport {
implicit lazy val system = ActorSystem("SharedCamelSystem", AkkaSpec.testConf)
implicit lazy val camel = CamelExtension(system)
abstract override protected def afterAll() {
abstract override protected def afterAll(): Unit = {
super.afterAll()
TestKit.shutdownActorSystem(system)
}
@ -61,13 +61,13 @@ private[camel] object TestSupport {
implicit var system: ActorSystem = _
implicit var camel: Camel = _
override protected def beforeEach() {
override protected def beforeEach(): Unit = {
super.beforeEach()
system = ActorSystem("NonSharedCamelSystem", AkkaSpec.testConf)
camel = CamelExtension(system)
}
override protected def afterEach() {
override protected def afterEach(): Unit = {
TestKit.shutdownActorSystem(system)
super.afterEach()
}

View file

@ -79,7 +79,7 @@ class UntypedProducerTest extends WordSpec with Matchers with BeforeAndAfterAll
object UntypedProducerTest {
class TestRoute extends RouteBuilder {
def configure {
def configure: Unit = {
from("direct:forward-test-1").to("mock:mock")
from("direct:producer-test-1").process(new Processor() {
def process(exchange: Exchange) = {

View file

@ -13,14 +13,14 @@ import akka.camel.internal.ActivationProtocol._
class ActivationTrackerTest extends TestKit(ActorSystem("ActivationTrackerTest")) with WordSpecLike with Matchers with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen {
override protected def afterAll() { shutdown() }
override protected def afterAll(): Unit = { shutdown() }
var actor: TestProbe = _
var awaiting: Awaiting = _
var anotherAwaiting: Awaiting = _
val cause = new Exception("cause of failure")
override protected def beforeEach() {
override protected def beforeEach(): Unit = {
actor = TestProbe()
awaiting = new Awaiting(actor)
anotherAwaiting = new Awaiting(actor)

View file

@ -113,7 +113,7 @@ class ActorProducerTest extends TestKit(ActorSystem("ActorProducerTest")) with W
"response is not sent by actor" must {
val latch = TestLatch(1)
val callback = new AsyncCallback {
def done(doneSync: Boolean) {
def done(doneSync: Boolean): Unit = {
latch.countDown()
}
}
@ -339,7 +339,7 @@ private[camel] trait ActorProducerFixture extends MockitoSugar with BeforeAndAft
var actorEndpointPath: ActorEndpointPath = _
var actorComponent: ActorComponent = _
override protected def beforeEach() {
override protected def beforeEach(): Unit = {
asyncCallback = createAsyncCallback
probe = TestProbe()
@ -380,7 +380,7 @@ private[camel] trait ActorProducerFixture extends MockitoSugar with BeforeAndAft
message = CamelMessage(null, null)
}
override protected def afterAll() {
override protected def afterAll(): Unit = {
shutdown()
}
@ -396,12 +396,12 @@ private[camel] trait ActorProducerFixture extends MockitoSugar with BeforeAndAft
class TestAsyncCallback extends AsyncCallback {
def expectNoCallWithin(duration: Duration): Unit =
if (callbackReceived.await(duration.length, duration.unit)) fail("NOT expected callback, but received one!")
def awaitCalled(timeout: Duration = 1 second) { valueWithin(1 second) }
def awaitCalled(timeout: Duration = 1 second): Unit = { valueWithin(1 second) }
val callbackReceived = new CountDownLatch(1)
val callbackValue = new AtomicBoolean()
def done(doneSync: Boolean) {
def done(doneSync: Boolean): Unit = {
callbackValue set doneSync
callbackReceived.countDown()
}
@ -422,7 +422,7 @@ private[camel] trait ActorProducerFixture extends MockitoSugar with BeforeAndAft
endpoint
}
def prepareMocks(actor: ActorRef, message: CamelMessage = message, outCapable: Boolean) {
def prepareMocks(actor: ActorRef, message: CamelMessage = message, outCapable: Boolean): Unit = {
when(actorEndpointPath.findActorIn(any[ActorSystem])) thenReturn Option(actor)
when(exchange.toRequestMessage(any[Map[String, Any]])) thenReturn message
when(exchange.isOutCapable) thenReturn outCapable