forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-5232] Introduce RobustActorSystem
The RobustActorSystem has a configurable UncaughtExceptionHandler. Per default it is started with the FatalExitExceptionHandler. This handler terminates the JVM whenever it sees an uncaught exception. This closes apache#6334.
- Loading branch information
1 parent
fb4eb93
commit 6fdeb5d
Showing
8 changed files
with
206 additions
and
10 deletions.
There are no files selected for viewing
98 changes: 98 additions & 0 deletions
98
flink-runtime/src/main/scala/akka/actor/RobustActorSystem.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,98 @@ | ||
/* | ||
* 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 | ||
* | ||
* https://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. | ||
*/ | ||
|
||
package akka.actor | ||
|
||
import java.lang.Thread.UncaughtExceptionHandler | ||
|
||
import akka.actor.ActorSystem.findClassLoader | ||
import akka.actor.setup.ActorSystemSetup | ||
import com.typesafe.config.{Config, ConfigFactory} | ||
import org.apache.flink.runtime.util.FatalExitExceptionHandler | ||
|
||
import scala.concurrent.ExecutionContext | ||
|
||
/** | ||
* [[ActorSystemImpl]] which has a configurable [[java.lang.Thread.UncaughtExceptionHandler]]. | ||
*/ | ||
class RobustActorSystem( | ||
name: String, | ||
applicationConfig: Config, | ||
classLoader: ClassLoader, | ||
defaultExecutionContext: Option[ExecutionContext], | ||
guardianProps: Option[Props], | ||
setup: ActorSystemSetup, | ||
val optionalUncaughtExceptionHandler: Option[UncaughtExceptionHandler]) | ||
extends ActorSystemImpl( | ||
name, | ||
applicationConfig, | ||
classLoader, | ||
defaultExecutionContext, | ||
guardianProps, | ||
setup) { | ||
|
||
override protected def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = | ||
optionalUncaughtExceptionHandler.getOrElse(super.uncaughtExceptionHandler) | ||
} | ||
|
||
object RobustActorSystem { | ||
def create(name: String, applicationConfig: Config): RobustActorSystem = { | ||
apply(name, ActorSystemSetup.create(BootstrapSetup(None, Option(applicationConfig), None))) | ||
} | ||
|
||
def create( | ||
name: String, | ||
applicationConfig: Config, | ||
uncaughtExceptionHandler: UncaughtExceptionHandler): RobustActorSystem = { | ||
apply( | ||
name, | ||
ActorSystemSetup.create(BootstrapSetup(None, Option(applicationConfig), None)), | ||
uncaughtExceptionHandler | ||
) | ||
} | ||
|
||
def apply(name: String, setup: ActorSystemSetup): RobustActorSystem = { | ||
internalApply(name, setup, Some(FatalExitExceptionHandler.INSTANCE)) | ||
} | ||
|
||
def apply( | ||
name: String, | ||
setup: ActorSystemSetup, | ||
uncaughtExceptionHandler: UncaughtExceptionHandler): RobustActorSystem = { | ||
internalApply(name, setup, Some(uncaughtExceptionHandler)) | ||
} | ||
|
||
def internalApply( | ||
name: String, | ||
setup: ActorSystemSetup, | ||
uncaughtExceptionHandler: Option[UncaughtExceptionHandler]): RobustActorSystem = { | ||
val bootstrapSettings = setup.get[BootstrapSetup] | ||
val cl = bootstrapSettings.flatMap(_.classLoader).getOrElse(findClassLoader()) | ||
val appConfig = bootstrapSettings.flatMap(_.config).getOrElse(ConfigFactory.load(cl)) | ||
val defaultEC = bootstrapSettings.flatMap(_.defaultExecutionContext) | ||
|
||
new RobustActorSystem( | ||
name, | ||
appConfig, | ||
cl, | ||
defaultEC, | ||
None, | ||
setup, | ||
uncaughtExceptionHandler).start() | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
98 changes: 98 additions & 0 deletions
98
flink-runtime/src/test/scala/akka/actor/RobustActorSystemTest.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,98 @@ | ||
/* | ||
* 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 | ||
* | ||
* https://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. | ||
*/ | ||
|
||
package akka.actor | ||
|
||
import java.lang.Thread.UncaughtExceptionHandler | ||
|
||
import org.apache.flink.runtime.akka.AkkaUtils | ||
import org.junit.{After, Before, Test} | ||
import org.scalatest.Matchers | ||
import org.scalatest.junit.JUnitSuite | ||
|
||
import scala.concurrent.duration.Duration | ||
import scala.concurrent.{Await, Future, Promise} | ||
import scala.util.Success | ||
|
||
class RobustActorSystemTest extends JUnitSuite with Matchers { | ||
|
||
var robustActorSystem: RobustActorSystem = null | ||
var testingUncaughtExceptionHandler: TestingUncaughtExceptionHandler = null | ||
|
||
@Before | ||
def setup(): Unit = { | ||
testingUncaughtExceptionHandler = new TestingUncaughtExceptionHandler | ||
robustActorSystem = RobustActorSystem.create( | ||
"testSystem", | ||
AkkaUtils.testDispatcherConfig, | ||
testingUncaughtExceptionHandler) | ||
} | ||
|
||
@After | ||
def teardown(): Unit = { | ||
robustActorSystem.terminate() | ||
testingUncaughtExceptionHandler = null; | ||
} | ||
|
||
@Test | ||
def testUncaughtExceptionHandler(): Unit = { | ||
val error = new UnknownError("Foobar") | ||
|
||
Future { | ||
throw error | ||
}(robustActorSystem.dispatcher) | ||
|
||
val caughtException = Await.result( | ||
testingUncaughtExceptionHandler.exceptionPromise.future, | ||
Duration.Inf) | ||
|
||
caughtException should equal (error) | ||
} | ||
|
||
@Test | ||
def testUncaughtExceptionHandlerFromActor(): Unit = { | ||
val error = new UnknownError() | ||
val actor = robustActorSystem.actorOf(Props.create(classOf[UncaughtExceptionActor], error)) | ||
|
||
actor ! Failure | ||
|
||
val caughtException = Await.result( | ||
testingUncaughtExceptionHandler.exceptionPromise.future, | ||
Duration.Inf) | ||
|
||
caughtException should equal (error) | ||
} | ||
} | ||
|
||
class TestingUncaughtExceptionHandler extends UncaughtExceptionHandler { | ||
val exceptionPromise: Promise[Throwable] = Promise() | ||
|
||
override def uncaughtException(t: Thread, e: Throwable): Unit = { | ||
exceptionPromise.complete(Success(e)) | ||
} | ||
} | ||
|
||
class UncaughtExceptionActor(failure: Throwable) extends Actor { | ||
override def receive: Receive = { | ||
case Failure => { | ||
throw failure | ||
}; | ||
} | ||
} | ||
|
||
case object Failure |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters