diff --git a/deploysettings.xml b/deploysettings.xml index e36d8480d47e66c970d714e9e82abd730a63009c..3b2670ddf87f518949333e59547e71759bb0849c 100644 --- a/deploysettings.xml +++ b/deploysettings.xml @@ -19,7 +19,7 @@ under the License. --> diff --git a/flink-addons/flink-avro/pom.xml b/flink-addons/flink-avro/pom.xml index 16a513769c187dd289c0985d9e20fb4e131403b8..1751cd1c947b22bc54b3c16d2763a223efa7d472 100644 --- a/flink-addons/flink-avro/pom.xml +++ b/flink-addons/flink-avro/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - diff --git a/flink-addons/flink-hadoop-compatibility/pom.xml b/flink-addons/flink-hadoop-compatibility/pom.xml index c1a19541d8cc6a20c34b52fa03418b9da2a10a28..49440d8e6ed7f44a1b27fc9c1486420655f7d293 100644 --- a/flink-addons/flink-hadoop-compatibility/pom.xml +++ b/flink-addons/flink-hadoop-compatibility/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - diff --git a/flink-addons/flink-hbase/pom.xml b/flink-addons/flink-hbase/pom.xml index 7dd89ceb7cd313d5e893b079d2512a2ac463f1b0..316fab6922ed682d5de3d98902f5be6f5aa921d8 100644 --- a/flink-addons/flink-hbase/pom.xml +++ b/flink-addons/flink-hbase/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-addons/flink-jdbc/pom.xml b/flink-addons/flink-jdbc/pom.xml index 6b56e2b198275fb1760c4801661eccd89cd4aa87..7022782679652c449fc6334fc2130a57bb627212 100644 --- a/flink-addons/flink-jdbc/pom.xml +++ b/flink-addons/flink-jdbc/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - diff --git a/flink-addons/flink-spargel/pom.xml b/flink-addons/flink-spargel/pom.xml index 219472283f436362fed0474b038cd4199d35127a..f9b64c01ff69d461724183c5b0ac9713eb3874ca 100644 --- a/flink-addons/flink-spargel/pom.xml +++ b/flink-addons/flink-spargel/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml index deded66b6d90fdad4b6bed3c677058d4971fe71b..6ea85c3b55a4aac2f1db91635536d9257a6ceec5 100644 --- a/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml +++ b/flink-addons/flink-streaming/flink-streaming-connectors/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-addons/flink-streaming/flink-streaming-core/pom.xml b/flink-addons/flink-streaming/flink-streaming-core/pom.xml index b2bbccc6402af485f8b5f2594e1e06b7d4bcdb2b..0d3e4eb35e5a98753f810ec97b8252c7d246948e 100644 --- a/flink-addons/flink-streaming/flink-streaming-core/pom.xml +++ b/flink-addons/flink-streaming/flink-streaming-core/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-addons/flink-streaming/flink-streaming-examples/pom.xml b/flink-addons/flink-streaming/flink-streaming-examples/pom.xml index 5ba8234305b153084a147b7393411d623dcc7279..95807b602d509cd40f7e44fe15fb82d6b2e7a321 100644 --- a/flink-addons/flink-streaming/flink-streaming-examples/pom.xml +++ b/flink-addons/flink-streaming/flink-streaming-examples/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-addons/flink-streaming/pom.xml b/flink-addons/flink-streaming/pom.xml index bfbea18732a47f86166979479467435d63da0ccc..b266ab2fda5bb9617c16267f4cb16a50c79d2a26 100644 --- a/flink-addons/flink-streaming/pom.xml +++ b/flink-addons/flink-streaming/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-addons/flink-yarn/pom.xml b/flink-addons/flink-yarn/pom.xml index c2bb418b5d2eb34a992548de49bbb95736d4fe43..fd4c2b5a2726f54816a19160d332d59d3c6b67b5 100644 --- a/flink-addons/flink-yarn/pom.xml +++ b/flink-addons/flink-yarn/pom.xml @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-addons/pom.xml b/flink-addons/pom.xml index be3cfa9f4bc1d8712f588b2ed8d6526b5b3ea70e..cfa7b74aec72d0ea51235d3e73a5cdb886b6ccc5 100644 --- a/flink-addons/pom.xml +++ b/flink-addons/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml index a7b75eeb754b88e47c344f14ca0061ea49801be5..dbb500554fd945de18c42a8f42f1183c0b742a93 100644 --- a/flink-clients/pom.xml +++ b/flink-clients/pom.xml @@ -18,7 +18,7 @@ specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-clients/src/main/java/org/apache/flink/client/minicluster/NepheleMiniCluster.java b/flink-clients/src/main/java/org/apache/flink/client/minicluster/NepheleMiniCluster.java index 79fe81c6ff68886b12a73eb0cee5256d13246127..8fd381c5872050047a1c87f080b64c91c2c33a67 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/minicluster/NepheleMiniCluster.java +++ b/flink-clients/src/main/java/org/apache/flink/client/minicluster/NepheleMiniCluster.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.instance.InstanceManager; import org.apache.flink.runtime.instance.LocalInstanceManager; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobManager; -import org.apache.flink.runtime.messages.JobmanagerMessages; +import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.taskmanager.TaskManager; import java.lang.reflect.Method; @@ -40,6 +40,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; +import org.apache.flink.runtime.util.EnvironmentInformation; import scala.concurrent.Await; import scala.concurrent.Future; @@ -225,8 +226,7 @@ public class NepheleMiniCluster { Configuration configuration = GlobalConfiguration.getConfiguration(); // start the job manager - jobManager = JobManager.startActorSystemAndActor("flink", HOSTNAME, jobManagerRpcPort, "jobmanager", - configuration); + jobManager = JobManager.startActorSystemAndActor(HOSTNAME, jobManagerRpcPort, configuration); int tmRPCPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT); @@ -237,8 +237,7 @@ public class NepheleMiniCluster { Configuration tmConfiguration = GlobalConfiguration.getConfiguration(); tmConfiguration.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, tmRPCPort + i); tmConfiguration.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, tmDataPort + i); - ActorSystem taskManager = TaskManager.startActorSystemAndActor("flink", HOSTNAME, tmRPCPort+i, - "taskmanager" + (i+1), configuration); + ActorSystem taskManager = TaskManager.startActorSystemAndActor(HOSTNAME, tmRPCPort+i, configuration); taskManagers.add(taskManager); } @@ -275,19 +274,22 @@ public class NepheleMiniCluster { // ------------------------------------------------------------------------ private void waitForJobManagerToBecomeReady(int numTaskManagers) throws Exception { + LOG.debug("Wait until " + numTaskManagers + " task managers are ready."); boolean notReady = true; - ActorSelection jobmanagerSelection = jobManager.actorSelection("jobmanager"); Timeout timeout = new Timeout(1L, TimeUnit.MINUTES); + ActorSelection jobManagerSelection = jobManager.actorSelection("/user/jobmanager"); while(notReady){ - Future futureNumTaskManagers = Patterns.ask(jobmanagerSelection, - JobmanagerMessages.RequestNumberRegisteredTaskManager$.MODULE$, timeout); + Future futureNumTaskManagers = Patterns.ask(jobManagerSelection, + JobManagerMessages.RequestNumberRegisteredTaskManager$.MODULE$, timeout); int numRegisteredTaskManagers = (Integer)Await.result(futureNumTaskManagers, timeout.duration()); + LOG.debug("Number of registered task manager: " + numRegisteredTaskManagers); + if(numRegisteredTaskManagers < numTaskManagers){ - Thread.sleep(50); + Thread.sleep(500); } // make sure that not just the jobmanager has the slots, but also the taskmanager diff --git a/flink-compiler/pom.xml b/flink-compiler/pom.xml index 2c33fd759425fa80ddb364c29bb85cb1ca377c92..ccd338e88e80ecd53527af0ccae1740031eb4aef 100644 --- a/flink-compiler/pom.xml +++ b/flink-compiler/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-core/pom.xml b/flink-core/pom.xml index 481c20e72ca81f2060c9658a2c8e0131e80e14c9..9d5a00e26a629990b5aabb3d15ef0c0e0947e896 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/OperatorInformation.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/OperatorInformation.java index 63d70f2c9dd88cff4765f139cdbd03d1b6664a1c..75ebc6248629a1c1cde47f1ed2ac23d666e6ccb6 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/OperatorInformation.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/OperatorInformation.java @@ -17,39 +17,132 @@ */ +import java.io.File +import java.net.InetAddress + import akka.actor._ -import org.apache.flink.configuration.Configuration +import org.apache.flink.configuration.{ConfigConstants, GlobalConfiguration, Configuration} +import org.apache.flink.runtime.ActorLogMessages import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.messages.JobmanagerMessages.RequestNumberRegisteredTaskManager +import org.apache.flink.runtime.executiongraph.{ExecutionGraph} +import org.apache.flink.runtime.instance.{InstanceManager} +import org.apache.flink.runtime.jobgraph.JobID +import org.apache.flink.runtime.jobmanager.accumulators.AccumulatorManager +import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler +import org.apache.flink.runtime.messages.EventCollectorMessages.RegisterArchiveListener +import org.apache.flink.runtime.messages.JobManagerMessages.{CancelJob, SubmitJob, RequestNumberRegisteredTaskManager} import org.apache.flink.runtime.messages.RegistrationMessages._ +import org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat +import org.apache.flink.runtime.profiling.ProfilingUtils +import org.apache.flink.runtime.profiling.impl.JobManagerProfilerImpl +import org.slf4j.LoggerFactory + +class JobManager(archiveCount: Int, profiling: Boolean, recommendedPollingInterval: Int) extends Actor with +ActorLogMessages with ActorLogging { + + val profiler = if(profiling){ + new JobManagerProfilerImpl(InetAddress.getByName(self.path.address.host.getOrElse("localhost"))) + }else{ + null + } + + // will be removed + val archive = context.actorOf(Props(classOf[MemoryArchivist], archiveCount), "archive") + val eventCollector = context.actorOf(Props(classOf[EventCollector], recommendedPollingInterval), "eventcollector") + + + val accumulatorManager = new AccumulatorManager(Math.min(1, archiveCount)) + val instanceManager = new InstanceManager() + val scheduler = new DefaultScheduler() + val webserver = null -import scala.collection.mutable + val currentJobs = scala.collection.concurrent.TrieMap[JobID, ExecutionGraph]() -class JobManager extends Actor with ActorLogging { + eventCollector ! RegisterArchiveListener(archive) - val taskManagers = new mutable.HashSet[ActorRef]() + instanceManager.addInstanceListener(scheduler) - override def receive: Receive = { - case RegisterTaskManager(hardwareInformation) => + override def postStop(): Unit = { + instanceManager.shutdown() + scheduler.shutdown() + } + + override def receiveWithLogMessages: Receive = { + case RegisterTaskManager(hardwareInformation, numberOfSlots) => val taskManager = sender() - taskManagers += taskManager + val instanceID = instanceManager.registerTaskManager(taskManager, hardwareInformation, numberOfSlots) context.watch(taskManager); - taskManager ! AcknowledgeRegistration + taskManager ! AcknowledgeRegistration(instanceID) case RequestNumberRegisteredTaskManager => - sender() ! taskManagers.size + sender() ! instanceManager.getNumberOfRegisteredTaskManagers + + case SubmitJob(jobGraph) => + + case CancelJob(jobID) => + + case Heartbeat(instanceID) => + instanceManager.reportHeartBeat(instanceID) + + } } object JobManager{ - def startActorSystemAndActor(systemName: String, hostname: String, port: Int, actorName: String, - configuration: Configuration): ActorSystem = { - val actorSystem = AkkaUtils.createActorSystem(systemName, hostname, port, configuration) - startActor(actorSystem, actorName) + val LOG = LoggerFactory.getLogger(classOf[JobManager]) + val FAILURE_RETURN_CODE = 1 + + def main(args: Array[String]):Unit = { + val (hostname, port, configuration) = initialize(args) + + val jobManagerSystem = startActorSystemAndActor(hostname, port, configuration) + jobManagerSystem.awaitTermination() + } + + def initialize(args: Array[String]):(String, Int, Configuration) = { + val parser = new scopt.OptionParser[JobManagerCLIConfiguration]("jobmanager"){ + head("flink jobmanager") + opt[String]("configDir") action { (x, c) => + c.copy(configDir = x) + } text("Specify configuration directory.") + } + + parser.parse(args, JobManagerCLIConfiguration()) map { + config => + GlobalConfiguration.loadConfiguration(config.configDir) + val configuration = GlobalConfiguration.getConfiguration() + if(config.configDir != null && new File(config.configDir).isDirectory){ + configuration.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, config.configDir + "/..") + } + + val hostname = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null) + val port = configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, + ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT) + + (hostname, port, configuration) + } getOrElse { + LOG.error("CLI Parsing failed. Usage: " + parser.usage) + sys.exit(FAILURE_RETURN_CODE) + } + } + + def startActorSystemAndActor(hostname: String, port: Int, configuration: Configuration): ActorSystem = { + val actorSystem = AkkaUtils.createActorSystem(hostname, port, configuration) + startActor(actorSystem, configuration) actorSystem } - def startActor(actorSystem: ActorSystem, actorName: String): ActorRef = { - actorSystem.actorOf(Props(classOf[JobManager]), actorName) + def startActor(actorSystem: ActorSystem, configuration: Configuration): ActorRef = { + val archiveCount = configuration.getInteger(ConfigConstants.JOB_MANAGER_WEB_ARCHIVE_COUNT, + ConfigConstants.DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT) + val profilingEnabled = configuration.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true) + val recommendedPollingInterval = configuration.getInteger(ConfigConstants.JOBCLIENT_POLLING_INTERVAL_KEY, + ConfigConstants.DEFAULT_JOBCLIENT_POLLING_INTERVAL) + + actorSystem.actorOf(Props(classOf[JobManager], archiveCount, profilingEnabled, recommendedPollingInterval), "jobmanager") + } + + def getAkkaURL(address: String): String = { + s"akka.tcp://flink@${address}/user/jobmanager" } } diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index 79a2557265fdb380ef6eed463cb1a0e10d256bc9..511c66ca7716483755b24817fe0f90225496a6c8 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index 0dc01965d4c6b255970619d4839e2b37b28a0275..0e56d3f4d3f028040e86ca7d5cfa6106aea0775d 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -18,7 +18,7 @@ under the License. --> bin diff --git a/flink-dist/src/main/assemblies/yarn-uberjar.xml b/flink-dist/src/main/assemblies/yarn-uberjar.xml index 54be9de0769bf9e9163e2d6d73b08c0cc9ce589e..db87f5192d9d6a57f1f56c3a5a91d96e2bcd9e0c 100644 --- a/flink-dist/src/main/assemblies/yarn-uberjar.xml +++ b/flink-dist/src/main/assemblies/yarn-uberjar.xml @@ -18,7 +18,7 @@ under the License. --> diff --git a/flink-dist/src/main/assemblies/yarn.xml b/flink-dist/src/main/assemblies/yarn.xml index 6cb7d3da1781dc75222595ad2cc586db55671513..189ac6fc5b4af569f5ce1389205a513ac4234039 100644 --- a/flink-dist/src/main/assemblies/yarn.xml +++ b/flink-dist/src/main/assemblies/yarn.xml @@ -19,7 +19,7 @@ under the License. yarn diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh index 02bd265a09db96e20e2be0df075c19bd1e9d5359..64128dff09745f14419dcd422fa93178244c05bd 100755 --- a/flink-dist/src/main/flink-bin/bin/config.sh +++ b/flink-dist/src/main/flink-bin/bin/config.sh @@ -191,7 +191,7 @@ JVM_ARGS="" CLASSPATH=`manglePathList $( echo $FLINK_LIB_DIR/*.jar . | sed 's/ /:/g' )` # Auxilliary function which extracts the name of host from a line which -# also potentialy includes topology information and the instance type +# also potentialy includes topology information and the taskManager type extractHostName() { # extract first part of string (before any whitespace characters) SLAVE=$1 diff --git a/flink-examples/flink-java-examples/pom.xml b/flink-examples/flink-java-examples/pom.xml index 0302a96bdacf6b5752ce508ec5911b9eb935bdeb..8fc772db069bf43ae8e7eddda7ec5d324aa52c6c 100644 --- a/flink-examples/flink-java-examples/pom.xml +++ b/flink-examples/flink-java-examples/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/flink-examples/flink-scala-examples/pom.xml b/flink-examples/flink-scala-examples/pom.xml index 9f13741808aa1b27f59dd357ff4db15f961da80f..37a598a99806d6eba4f7a472ea0a645886517d8a 100644 --- a/flink-examples/flink-scala-examples/pom.xml +++ b/flink-examples/flink-scala-examples/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index e4c0d25210c9cac7296f5147ffe65a3bb4387f20..575bf54536d965616388a2dd48d1f073a4a15605 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/flink-java/pom.xml b/flink-java/pom.xml index 6865cc9c489cd0bb5e93b10bc229887c3c909cf8..070bb83d65a0e0a81c6b38fe09d2a8edc8852d2b 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-java8/pom.xml b/flink-java8/pom.xml index af9b65cbac6a10297b6b019ad6710602eb262ef7..f175a7353e5537f0fe2f26d6f114e263f7d7cd5b 100644 --- a/flink-java8/pom.xml +++ b/flink-java8/pom.xml @@ -18,7 +18,7 @@ specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-quickstart/flink-quickstart-java/pom.xml b/flink-quickstart/flink-quickstart-java/pom.xml index fc72d4c5543130a51177a491026f892fa67b2c58..e5f09b919d1904af84b8123351d9fb55e988d576 100644 --- a/flink-quickstart/flink-quickstart-java/pom.xml +++ b/flink-quickstart/flink-quickstart-java/pom.xml @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index d26f926e40f1887b7a48ec6e975210cdb189ba43..88ea561c73da9ece3bdd968498f4dd6eb419e496 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-quickstart/flink-quickstart-scala/pom.xml b/flink-quickstart/flink-quickstart-scala/pom.xml index bad553cc2e472a7ba6d44566417ee443b553eb8a..a7011e6fa9cf805688ac6bc49e0defe68d6088d0 100644 --- a/flink-quickstart/flink-quickstart-scala/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/pom.xml @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-quickstart/pom.xml b/flink-quickstart/pom.xml index 741ad0feef49a8c75dad45b298f86870ef605f5b..7b3f8b51a1ba60ed2f139854af32fc7097adb592 100644 --- a/flink-quickstart/pom.xml +++ b/flink-quickstart/pom.xml @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 5a87b90b388bbdce68d5b1d217ee52b2122b76f9..520927f106ad1f32ce723383c6396ee073603184 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 @@ -114,6 +114,18 @@ under the License. akka-remote_2.10 2.3.5 + + + com.typesafe.akka + akka-slf4j_2.10 + 2.3.5 + + + + com.github.scopt + scopt_2.10 + 3.2.0 + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 29f157c67c5e73d921339ad3a65098cd4fdc54ee..1e048dbd7d47ea8cc13a2715def9a1b5dabd2ca8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -48,7 +48,8 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; +import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler; +import org.apache.flink.runtime.messages.ExecutionGraphMessages; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.util.ExceptionUtils; @@ -100,6 +101,10 @@ public class ExecutionGraph { private final List jobStatusListeners; private final List executionListeners; + + private final List jobStatusListenerActors; + + private final List executionListenerActors; private final long[] stateTimestamps; @@ -150,6 +155,8 @@ public class ExecutionGraph { this.jobStatusListeners = new CopyOnWriteArrayList(); this.executionListeners = new CopyOnWriteArrayList(); + this.jobStatusListenerActors = new CopyOnWriteArrayList(); + this.executionListenerActors = new CopyOnWriteArrayList(); this.stateTimestamps = new long[JobStatus.values().length]; this.stateTimestamps[JobStatus.CREATED.ordinal()] = System.currentTimeMillis(); @@ -315,7 +322,23 @@ public class ExecutionGraph { } // -------------------------------------------------------------------------------------------- - // Actions + + public void registerJobStatusListener(JobStatusListener jobStatusListener) { + this.jobStatusListeners.add(jobStatusListener); + } + + public void registerExecutionListener(ExecutionListener executionListener) { + this.executionListeners.add(executionListener); + } + + public void registerJobStatusListener(ActorRef listener){ + this.jobStatusListenerActors.add(listener); + } + + public void registerExecutionListener(ActorRef listener){ + this.executionListenerActors.add(listener); + } + // -------------------------------------------------------------------------------------------- public void scheduleForExecution(Scheduler scheduler) throws JobException { @@ -682,6 +705,10 @@ public class ExecutionGraph { } } } + + for(ActorRef listener: jobStatusListenerActors){ + listener.tell(new ExecutionGraphMessages.JobStatusChanged(this, newState, message), ActorRef.noSender()); + } } /** @@ -692,19 +719,10 @@ public class ExecutionGraph { * @param newExecutionState * @param error */ - void notifyExecutionChange(JobVertexID vertexId, int subtask, ExecutionAttemptID executionId, ExecutionState newExecutionState, Throwable error) { - - // we must be very careful here with exceptions - if (this.executionListeners.size() > 0) { - - String message = error == null ? null : ExceptionUtils.stringifyException(error); - for (ExecutionListener listener : this.executionListeners) { - try { - listener.executionStateChanged(jobID, vertexId, subtask, executionId, newExecutionState, message); - } - catch (Throwable t) { - LOG.error("Notification of execution state change caused an error.", t); - } + void notifyExecutionChange(JobVertexID vertexId, int subtask, ExecutionState2 newExecutionState, String optionalMessage) { + for (ExecutionListener listener : this.executionListeners) { + try { + listener.executionStateChanged(jobID, vertexId, subtask,null, newExecutionState, optionalMessage); } } @@ -760,3 +778,1102 @@ public class ExecutionGraph { } } } + +// +// /** +// * Applies the user defined settings to the execution graph. +// * +// * @param temporaryGroupVertexMap +// * mapping between job vertices and the corresponding group vertices. +// * @throws GraphConversionException +// * thrown if an error occurs while applying the user settings. +// */ +// private void applyUserDefinedSettings(final HashMap temporaryGroupVertexMap) +// throws GraphConversionException { +// +// // The check for cycles in the dependency chain for instance sharing is already checked in +// // submitJob method of the job manager +// +// // If there is no cycle, apply the settings to the corresponding group vertices +// final Iterator> it = temporaryGroupVertexMap.entrySet() +// .iterator(); +// while (it.hasNext()) { +// +// final Map.Entry entry = it.next(); +// final AbstractJobVertex jobVertex = entry.getKey(); +// if (jobVertex.getVertexToShareInstancesWith() != null) { +// +// final AbstractJobVertex vertexToShareInstancesWith = jobVertex.getVertexToShareInstancesWith(); +// final ExecutionGroupVertex groupVertex = entry.getValue(); +// final ExecutionGroupVertex groupVertexToShareInstancesWith = temporaryGroupVertexMap +// .get(vertexToShareInstancesWith); +// groupVertex.shareInstancesWith(groupVertexToShareInstancesWith); +// } +// } +// +// // Second, we create the number of execution vertices each group vertex is supposed to manage +// Iterator it2 = new ExecutionGroupVertexIterator(this, true, -1); +// while (it2.hasNext()) { +// +// final ExecutionGroupVertex groupVertex = it2.next(); +// if (groupVertex.isNumberOfMembersUserDefined()) { +// groupVertex.createInitialExecutionVertices(groupVertex.getUserDefinedNumberOfMembers()); +// } +// } +// +// // Finally, apply the channel settings channel settings +// it2 = new ExecutionGroupVertexIterator(this, true, -1); +// while (it2.hasNext()) { +// +// final ExecutionGroupVertex groupVertex = it2.next(); +// for (int i = 0; i < groupVertex.getNumberOfForwardLinks(); i++) { +// +// final ExecutionGroupEdge edge = groupVertex.getForwardEdge(i); +// if (edge.isChannelTypeUserDefined()) { +// edge.changeChannelType(edge.getChannelType()); +// } +// +// // Create edges between execution vertices +// createExecutionEdgesForGroupEdge(edge); +// } +// } +// +// // Repair the instance assignment after having changed the channel types +// repairInstanceAssignment(); +// +// // Repair the instance sharing among different group vertices +// repairInstanceSharing(); +// +// // Finally, repair the stages +// repairStages(); +// } +// +// /** +// * Sets up an execution graph from a job graph. +// * +// * @param jobGraph +// * the job graph to create the execution graph from +// * @param defaultParallelism +// * defaultParallelism in case that nodes have no parallelism set +// * @throws GraphConversionException +// * thrown if the job graph is not valid and no execution graph can be constructed from it +// */ +// private void constructExecutionGraph(final JobGraph jobGraph, final int defaultParallelism) +// throws GraphConversionException { +// +// // Clean up temporary data structures +// final HashMap temporaryVertexMap = new HashMap(); +// final HashMap temporaryGroupVertexMap = new HashMap(); +// +// // Initially, create only one execution stage that contains all group vertices +// final ExecutionStage initialExecutionStage = new ExecutionStage(this, 0); +// this.stages.add(initialExecutionStage); +// +// // Convert job vertices to execution vertices and initialize them +// final AbstractJobVertex[] all = jobGraph.getAllJobVertices(); +// for (int i = 0; i < all.length; i++) { +// if(all[i].getNumberOfSubtasks() == -1){ +// all[i].setNumberOfSubtasks(defaultParallelism); +// } +// +// final ExecutionVertex createdVertex = createVertex(all[i], initialExecutionStage); +// temporaryVertexMap.put(all[i], createdVertex); +// temporaryGroupVertexMap.put(all[i], createdVertex.getGroupVertex()); +// } +// +// // Create initial edges between the vertices +// createInitialGroupEdges(temporaryVertexMap); +// +// // Now that an initial graph is built, apply the user settings +// applyUserDefinedSettings(temporaryGroupVertexMap); +// +// // Calculate the connection IDs +// calculateConnectionIDs(); +// +// // Finally, construct the execution pipelines +// reconstructExecutionPipelines(); +// } +// +// private void createExecutionEdgesForGroupEdge(final ExecutionGroupEdge groupEdge) { +// +// final ExecutionGroupVertex source = groupEdge.getSourceVertex(); +// final int indexOfOutputGate = groupEdge.getIndexOfOutputGate(); +// final ExecutionGroupVertex target = groupEdge.getTargetVertex(); +// final int indexOfInputGate = groupEdge.getIndexOfInputGate(); +// +// final Map> inputChannelMap = new HashMap>(); +// +// // Unwire the respective gate of the source vertices +// final int currentNumberOfSourceNodes = source.getCurrentNumberOfGroupMembers(); +// for (int i = 0; i < currentNumberOfSourceNodes; ++i) { +// +// final ExecutionVertex sourceVertex = source.getGroupMember(i); +// final ExecutionGate outputGate = sourceVertex.getOutputGate(indexOfOutputGate); +// if (outputGate == null) { +// throw new IllegalStateException("wire: " + sourceVertex.getName() +// + " has no output gate with index " + indexOfOutputGate); +// } +// +// if (outputGate.getNumberOfEdges() > 0) { +// throw new IllegalStateException("wire: wire called on source " + sourceVertex.getName() + " (" + i +// + "), but number of output channels is " + outputGate.getNumberOfEdges() + "!"); +// } +// +// final int currentNumberOfTargetNodes = target.getCurrentNumberOfGroupMembers(); +// final List outputChannels = new ArrayList(); +// +// for (int j = 0; j < currentNumberOfTargetNodes; ++j) { +// +// final ExecutionVertex targetVertex = target.getGroupMember(j); +// final ExecutionGate inputGate = targetVertex.getInputGate(indexOfInputGate); +// if (inputGate == null) { +// throw new IllegalStateException("wire: " + targetVertex.getName() +// + " has no input gate with index " + indexOfInputGate); +// } +// +// if (inputGate.getNumberOfEdges() > 0 && i == 0) { +// throw new IllegalStateException("wire: wire called on target " + targetVertex.getName() + " (" +// + j + "), but number of input channels is " + inputGate.getNumberOfEdges() + "!"); +// } +// +// // Check if a wire is supposed to be created +// if (DistributionPatternProvider.createWire(groupEdge.getDistributionPattern(), +// i, j, currentNumberOfSourceNodes, currentNumberOfTargetNodes)) { +// +// final ChannelID outputChannelID = new ChannelID(); +// final ChannelID inputChannelID = new ChannelID(); +// +// final ExecutionEdge edge = new ExecutionEdge(outputGate, inputGate, groupEdge, outputChannelID, +// inputChannelID, outputGate.getNumberOfEdges(), inputGate.getNumberOfEdges()); +// +// this.edgeMap.put(outputChannelID, edge); +// this.edgeMap.put(inputChannelID, edge); +// +// outputChannels.add(edge); +// +// List inputChannels = inputChannelMap.get(inputGate.getGateID()); +// if (inputChannels == null) { +// inputChannels = new ArrayList(); +// inputChannelMap.put(inputGate.getGateID(), inputChannels); +// } +// +// inputChannels.add(edge); +// } +// } +// +// outputGate.replaceAllEdges(outputChannels); +// } +// +// // Finally, set the channels for the input gates +// final int currentNumberOfTargetNodes = target.getCurrentNumberOfGroupMembers(); +// for (int i = 0; i < currentNumberOfTargetNodes; ++i) { +// +// final ExecutionVertex targetVertex = target.getGroupMember(i); +// final ExecutionGate inputGate = targetVertex.getInputGate(indexOfInputGate); +// +// final List inputChannels = inputChannelMap.get(inputGate.getGateID()); +// if (inputChannels == null) { +// LOG.error("Cannot find input channels for gate ID " + inputGate.getGateID()); +// continue; +// } +// +// inputGate.replaceAllEdges(inputChannels); +// } +// +// } +// +// /** +// * Creates the initial edges between the group vertices +// * +// * @param vertexMap +// * the temporary vertex map +// * @throws GraphConversionException +// * if the initial wiring cannot be created +// */ +// private void createInitialGroupEdges(final HashMap vertexMap) +// throws GraphConversionException { +// +// Iterator> it = vertexMap.entrySet().iterator(); +// +// while (it.hasNext()) { +// +// final Map.Entry entry = it.next(); +// final AbstractJobVertex sjv = entry.getKey(); +// final ExecutionVertex sev = entry.getValue(); +// final ExecutionGroupVertex sgv = sev.getGroupVertex(); +// +// // First, build the group edges +// for (int i = 0; i < sjv.getNumberOfForwardConnections(); ++i) { +// final JobEdge edge = sjv.getForwardConnection(i); +// final AbstractJobVertex tjv = edge.getConnectedVertex(); +// +// final ExecutionVertex tev = vertexMap.get(tjv); +// final ExecutionGroupVertex tgv = tev.getGroupVertex(); +// // Use NETWORK as default channel type if nothing else is defined by the user +// ChannelType channelType = edge.getChannelType(); +// boolean userDefinedChannelType = true; +// if (channelType == null) { +// userDefinedChannelType = false; +// channelType = ChannelType.NETWORK; +// } +// +// final DistributionPattern distributionPattern = edge.getDistributionPattern(); +// +// // Connect the corresponding group vertices and copy the user settings from the job edge +// final ExecutionGroupEdge groupEdge = sgv.wireTo(tgv, edge.getIndexOfInputGate(), i, channelType, +// userDefinedChannelType,distributionPattern); +// +// final ExecutionGate outputGate = new ExecutionGate(new GateID(), sev, groupEdge, false); +// sev.insertOutputGate(i, outputGate); +// final ExecutionGate inputGate = new ExecutionGate(new GateID(), tev, groupEdge, true); +// tev.insertInputGate(edge.getIndexOfInputGate(), inputGate); +// } +// } +// } +// +// /** +// * Creates an execution vertex from a job vertex. +// * +// * @param jobVertex +// * the job vertex to create the execution vertex from +// * @param initialExecutionStage +// * the initial execution stage all group vertices are added to +// * @return the new execution vertex +// * @throws GraphConversionException +// * thrown if the job vertex is of an unknown subclass +// */ +// private ExecutionVertex createVertex(final AbstractJobVertex jobVertex, final ExecutionStage initialExecutionStage) +// throws GraphConversionException { +// +// // Create an initial execution vertex for the job vertex +// final Class invokableClass = jobVertex.getInvokableClass(); +// if (invokableClass == null) { +// throw new GraphConversionException("JobVertex " + jobVertex.getID() + " (" + jobVertex.getName() +// + ") does not specify a task"); +// } +// +// // Calculate the cryptographic signature of this vertex +// final ExecutionSignature signature = ExecutionSignature.createSignature(jobVertex.getInvokableClass(), +// jobVertex.getJobGraph().getJobID()); +// +// // Create a group vertex for the job vertex +// +// ExecutionGroupVertex groupVertex = null; +// try { +// groupVertex = new ExecutionGroupVertex(jobVertex.getName(), jobVertex.getID(), this, +// jobVertex.getNumberOfSubtasks(), jobVertex.getVertexToShareInstancesWith() != null ? true +// : false, 0, jobVertex.getConfiguration(), signature, +// invokableClass); +// } catch (Throwable t) { +// throw new GraphConversionException(t); +// } +// +// // Register input and output vertices separately +// if (jobVertex instanceof AbstractJobInputVertex) { +// +// final AbstractJobInputVertex jobInputVertex = (AbstractJobInputVertex) jobVertex; +// +// if (jobVertex instanceof InputFormatVertex) { +// try { +// // get a handle to the user code class loader +// ClassLoader cl = LibraryCacheManager.getClassLoader(jobVertex.getJobGraph().getJobID()); +// +// ((InputFormatVertex) jobVertex).initializeInputFormatFromTaskConfig(cl); +// } +// catch (Throwable t) { +// throw new GraphConversionException("Could not deserialize input format.", t); +// } +// } +// +// final Class inputSplitType = jobInputVertex.getInputSplitType(); +// +// InputSplit[] inputSplits; +// +// try { +// inputSplits = jobInputVertex.getInputSplits(jobVertex.getNumberOfSubtasks()); +// } +// catch (Throwable t) { +// throw new GraphConversionException("Cannot compute input splits for " + groupVertex.getName(), t); +// } +// +// if (inputSplits == null) { +// inputSplits = new InputSplit[0]; +// } +// +// LOG.info("Job input vertex " + jobVertex.getName() + " generated " + inputSplits.length + " input splits"); +// +// // assign input splits and type +// groupVertex.setInputSplits(inputSplits); +// groupVertex.setInputSplitType(inputSplitType); +// } +// +// if (jobVertex instanceof OutputFormatVertex){ +// final OutputFormatVertex jobOutputVertex = (OutputFormatVertex) jobVertex; +// +// try { +// // get a handle to the user code class loader +// ClassLoader cl = LibraryCacheManager.getClassLoader(jobVertex.getJobGraph().getJobID()); +// jobOutputVertex.initializeOutputFormatFromTaskConfig(cl); +// } +// catch (Throwable t) { +// throw new GraphConversionException("Could not deserialize output format.", t); +// } +// +// OutputFormat outputFormat = jobOutputVertex.getOutputFormat(); +// if (outputFormat != null && outputFormat instanceof InitializeOnMaster){ +// try { +// ((InitializeOnMaster) outputFormat).initializeGlobal(jobVertex.getNumberOfSubtasks()); +// } +// catch (Throwable t) { +// throw new GraphConversionException(t); +// } +// } +// } +// +// // Add group vertex to initial execution stage +// initialExecutionStage.addStageMember(groupVertex); +// +// final ExecutionVertex ev = new ExecutionVertex(this, groupVertex, jobVertex.getNumberOfForwardConnections(), +// jobVertex.getNumberOfBackwardConnections()); +// +// // Assign initial instance to vertex (may be overwritten later on when user settings are applied) +// ev.setAllocatedResource(new AllocatedResource(DummyInstance.createDummyInstance(), null)); +// +// return ev; +// } +// +// /** +// * Returns the number of input vertices registered with this execution graph. +// * +// * @return the number of input vertices registered with this execution graph +// */ +// public int getNumberOfInputVertices() { +// return this.stages.get(0).getNumberOfInputExecutionVertices(); +// } +// +// /** +// * Returns the number of input vertices for the given stage. +// * +// * @param stage +// * the index of the execution stage +// * @return the number of input vertices for the given stage +// */ +// public int getNumberOfInputVertices(int stage) { +// if (stage >= this.stages.size()) { +// return 0; +// } +// +// return this.stages.get(stage).getNumberOfInputExecutionVertices(); +// } +// +// /** +// * Returns the number of output vertices registered with this execution graph. +// * +// * @return the number of output vertices registered with this execution graph +// */ +// public int getNumberOfOutputVertices() { +// return this.stages.get(0).getNumberOfOutputExecutionVertices(); +// } +// +// /** +// * Returns the number of output vertices for the given stage. +// * +// * @param stage +// * the index of the execution stage +// * @return the number of input vertices for the given stage +// */ +// public int getNumberOfOutputVertices(final int stage) { +// if (stage >= this.stages.size()) { +// return 0; +// } +// +// return this.stages.get(stage).getNumberOfOutputExecutionVertices(); +// } +// +// /** +// * Returns the input vertex with the specified index. +// * +// * @param index +// * the index of the input vertex to return +// * @return the input vertex with the specified index or null if no input vertex with such an index +// * exists +// */ +// public ExecutionVertex getInputVertex(final int index) { +// return this.stages.get(0).getInputExecutionVertex(index); +// } +// +// /** +// * Returns the output vertex with the specified index. +// * +// * @param index +// * the index of the output vertex to return +// * @return the output vertex with the specified index or null if no output vertex with such an index +// * exists +// */ +// public ExecutionVertex getOutputVertex(final int index) { +// return this.stages.get(0).getOutputExecutionVertex(index); +// } +// +// /** +// * Returns the input vertex with the specified index for the given stage +// * +// * @param stage +// * the index of the stage +// * @param index +// * the index of the input vertex to return +// * @return the input vertex with the specified index or null if no input vertex with such an index +// * exists in that stage +// */ +// public ExecutionVertex getInputVertex(final int stage, final int index) { +// try { +// final ExecutionStage s = this.stages.get(stage); +// if (s == null) { +// return null; +// } +// +// return s.getInputExecutionVertex(index); +// +// } catch (ArrayIndexOutOfBoundsException e) { +// return null; +// } +// } +// +// /** +// * Returns the output vertex with the specified index for the given stage. +// * +// * @param stage +// * the index of the stage +// * @param index +// * the index of the output vertex to return +// * @return the output vertex with the specified index or null if no output vertex with such an index +// * exists in that stage +// */ +// public ExecutionVertex getOutputVertex(final int stage, final int index) { +// try { +// final ExecutionStage s = this.stages.get(stage); +// if (s == null) { +// return null; +// } +// +// return s.getOutputExecutionVertex(index); +// +// } catch (ArrayIndexOutOfBoundsException e) { +// return null; +// } +// } +// +// /** +// * Identifies an execution by the specified channel ID and returns it. +// * +// * @param id +// * the channel ID to identify the vertex with +// * @return the execution vertex which has a channel with ID id or null if no such vertex +// * exists in the execution graph +// */ +// public ExecutionVertex getVertexByChannelID(final ChannelID id) { +// final ExecutionEdge edge = this.edgeMap.get(id); +// if (edge == null) { +// return null; +// } +// +// if (id.equals(edge.getOutputChannelID())) { +// return edge.getOutputGate().getVertex(); +// } +// +// return edge.getInputGate().getVertex(); +// } +// +// /** +// * Finds an {@link ExecutionEdge} by its ID and returns it. +// * +// * @param id +// * the channel ID to identify the edge +// * @return the edge whose ID matches id or null if no such edge is known +// */ +// public ExecutionEdge getEdgeByID(final ChannelID id) { +// return this.edgeMap.get(id); +// } +// +// /** +// * Registers an execution vertex with the execution graph. +// * +// * @param vertex +// * the execution vertex to register +// */ +// void registerExecutionVertex(final ExecutionVertex vertex) { +// if (this.vertexMap.put(vertex.getID(), vertex) != null) { +// throw new IllegalStateException("There is already an execution vertex with ID " + vertex.getID() +// + " registered"); +// } +// } +// +// /** +// * Returns the execution vertex with the given vertex ID. +// * +// * @param id +// * the vertex ID to retrieve the execution vertex +// * @return the execution vertex matching the provided vertex ID or null if no such vertex could be +// * found +// */ +// public ExecutionVertex getVertexByID(final ExecutionVertexID id) { +// return this.vertexMap.get(id); +// } +// +// /** +// * Checks if the current execution stage has been successfully completed, i.e. +// * all vertices in this stage have successfully finished their execution. +// * +// * @return true if stage is completed, false otherwise +// */ +// private boolean isCurrentStageCompleted() { +// if (this.indexToCurrentExecutionStage >= this.stages.size()) { +// return true; +// } +// +// final ExecutionGraphIterator it = new ExecutionGraphIterator(this, this.indexToCurrentExecutionStage, true, +// true); +// while (it.hasNext()) { +// final ExecutionVertex vertex = it.next(); +// if (vertex.getExecutionState() != ExecutionState.FINISHED) { +// return false; +// } +// } +// +// return true; +// } +// +// /** +// * Checks if the execution of execution graph is finished. +// * +// * @return true if the execution of the graph is finished, false otherwise +// */ +// public boolean isExecutionFinished() { +// return (getJobStatus() == InternalJobStatus.FINISHED); +// } +// +// /** +// * Returns the job ID of the job configuration this execution graph was originally constructed from. +// * +// * @return the job ID of the job configuration this execution graph was originally constructed from +// */ +// public JobID getJobID() { +// return this.jobID; +// } +// +// /** +// * Returns the index of the current execution stage. +// * +// * @return the index of the current execution stage +// */ +// public int getIndexOfCurrentExecutionStage() { +// return this.indexToCurrentExecutionStage; +// } +// +// /** +// * Returns the stage which is currently executed. +// * +// * @return the currently executed stage or null if the job execution is already completed +// */ +// public ExecutionStage getCurrentExecutionStage() { +// +// try { +// return this.stages.get(this.indexToCurrentExecutionStage); +// } catch (ArrayIndexOutOfBoundsException e) { +// return null; +// } +// } +// +// public void repairStages() { +// +// final Map stageNumbers = new HashMap(); +// ExecutionGroupVertexIterator it = new ExecutionGroupVertexIterator(this, true, -1); +// +// while (it.hasNext()) { +// +// final ExecutionGroupVertex groupVertex = it.next(); +// int precedingNumber = 0; +// if (stageNumbers.containsKey(groupVertex)) { +// precedingNumber = stageNumbers.get(groupVertex).intValue(); +// } else { +// stageNumbers.put(groupVertex, Integer.valueOf(precedingNumber)); +// } +// +// for (int i = 0; i < groupVertex.getNumberOfForwardLinks(); i++) { +// +// final ExecutionGroupEdge edge = groupVertex.getForwardEdge(i); +// if (!stageNumbers.containsKey(edge.getTargetVertex())) { +// // Target vertex has not yet been discovered +// // Same stage as preceding vertex +// stageNumbers.put(edge.getTargetVertex(), Integer.valueOf(precedingNumber)); +// } else { +// final int stageNumber = stageNumbers.get(edge.getTargetVertex()).intValue(); +// if (stageNumber != precedingNumber) { +// stageNumbers.put(edge.getTargetVertex(), (int) Math.max(precedingNumber, stageNumber)); +// } +// } +// } +// } +// +// // Traverse the graph backwards (starting from the output vertices) to make sure vertices are allocated in a +// // stage as high as possible +// it = new ExecutionGroupVertexIterator(this, false, -1); +// +// while (it.hasNext()) { +// +// final ExecutionGroupVertex groupVertex = it.next(); +// final int succeedingNumber = stageNumbers.get(groupVertex); +// +// for (int i = 0; i < groupVertex.getNumberOfBackwardLinks(); i++) { +// +// final ExecutionGroupEdge edge = groupVertex.getBackwardEdge(i); +// final int stageNumber = stageNumbers.get(edge.getSourceVertex()); +// if (stageNumber != succeedingNumber) { +// throw new IllegalStateException(edge.getSourceVertex() + " and " + edge.getTargetVertex() +// + " are assigned to different stages"); +// } +// } +// } +// +// // Finally, assign the new stage numbers +// this.stages.clear(); +// final Iterator> it2 = stageNumbers.entrySet().iterator(); +// while (it2.hasNext()) { +// +// final Map.Entry entry = it2.next(); +// final ExecutionGroupVertex groupVertex = entry.getKey(); +// final int stageNumber = entry.getValue().intValue(); +// // Prevent out of bounds exceptions +// while (this.stages.size() <= stageNumber) { +// this.stages.add(null); +// } +// ExecutionStage executionStage = this.stages.get(stageNumber); +// // If the stage not yet exists, +// if (executionStage == null) { +// executionStage = new ExecutionStage(this, stageNumber); +// this.stages.set(stageNumber, executionStage); +// } +// +// executionStage.addStageMember(groupVertex); +// groupVertex.setExecutionStage(executionStage); +// } +// } +// +// public void repairInstanceSharing() { +// +// final Set availableResources = new LinkedHashSet(); +// +// final Iterator it = new ExecutionGroupVertexIterator(this, true, -1); +// while (it.hasNext()) { +// final ExecutionGroupVertex groupVertex = it.next(); +// if (groupVertex.getVertexToShareInstancesWith() == null) { +// availableResources.clear(); +// groupVertex.repairInstanceSharing(availableResources); +// } +// } +// } +// +// public void repairInstanceAssignment() { +// +// Iterator it = new ExecutionGraphIterator(this, true); +// while (it.hasNext()) { +// +// final ExecutionVertex sourceVertex = it.next(); +// +// for (int i = 0; i < sourceVertex.getNumberOfOutputGates(); ++i) { +// +// final ExecutionGate outputGate = sourceVertex.getOutputGate(i); +// final ChannelType channelType = outputGate.getChannelType(); +// if (channelType == ChannelType.IN_MEMORY) { +// final int numberOfOutputChannels = outputGate.getNumberOfEdges(); +// for (int j = 0; j < numberOfOutputChannels; ++j) { +// final ExecutionEdge outputChannel = outputGate.getEdge(j); +// outputChannel.getInputGate().getVertex() +// .setAllocatedResource(sourceVertex.getAllocatedResource()); +// } +// } +// } +// } +// +// it = new ExecutionGraphIterator(this, false); +// while (it.hasNext()) { +// +// final ExecutionVertex targetVertex = it.next(); +// +// for (int i = 0; i < targetVertex.getNumberOfInputGates(); ++i) { +// +// final ExecutionGate inputGate = targetVertex.getInputGate(i); +// final ChannelType channelType = inputGate.getChannelType(); +// if (channelType == ChannelType.IN_MEMORY) { +// final int numberOfInputChannels = inputGate.getNumberOfEdges(); +// for (int j = 0; j < numberOfInputChannels; ++j) { +// final ExecutionEdge inputChannel = inputGate.getEdge(j); +// inputChannel.getOutputGate().getVertex() +// .setAllocatedResource(targetVertex.getAllocatedResource()); +// } +// } +// } +// } +// } +// +// public ChannelType getChannelType(final ExecutionVertex sourceVertex, final ExecutionVertex targetVertex) { +// +// final ExecutionGroupVertex sourceGroupVertex = sourceVertex.getGroupVertex(); +// final ExecutionGroupVertex targetGroupVertex = targetVertex.getGroupVertex(); +// +// final List edges = sourceGroupVertex.getForwardEdges(targetGroupVertex); +// if (edges.size() == 0) { +// return null; +// } +// +// // On a task level, the two vertices are connected +// final ExecutionGroupEdge edge = edges.get(0); +// +// // Now lets see if these two concrete subtasks are connected +// final ExecutionGate outputGate = sourceVertex.getOutputGate(edge.getIndexOfOutputGate()); +// for (int i = 0; i < outputGate.getNumberOfEdges(); ++i) { +// +// final ExecutionEdge outputChannel = outputGate.getEdge(i); +// if (targetVertex == outputChannel.getInputGate().getVertex()) { +// return edge.getChannelType(); +// } +// } +// +// return null; +// } +// +// /** +// * Returns the job configuration that was originally attached to the job graph. +// * +// * @return the job configuration that was originally attached to the job graph +// */ +// public Configuration getJobConfiguration() { +// return this.jobConfiguration; +// } +// +// /** +// * Checks whether the job represented by the execution graph has the status FINISHED. +// * +// * @return true if the job has the status CREATED, false otherwise +// */ +// private boolean jobHasFinishedStatus() { +// +// final Iterator it = new ExecutionGraphIterator(this, true); +// +// while (it.hasNext()) { +// +// if (it.next().getExecutionState() != ExecutionState.FINISHED) { +// return false; +// } +// } +// +// return true; +// } +// +// /** +// * Checks whether the job represented by the execution graph has the status SCHEDULED. +// * +// * @return true if the job has the status SCHEDULED, false otherwise +// */ +// private boolean jobHasScheduledStatus() { +// +// final Iterator it = new ExecutionGraphIterator(this, true); +// +// while (it.hasNext()) { +// +// final ExecutionState s = it.next().getExecutionState(); +// if (s != ExecutionState.CREATED && s != ExecutionState.SCHEDULED && s != ExecutionState.READY) { +// return false; +// } +// } +// +// return true; +// } +// +// /** +// * Checks whether the job represented by the execution graph has the status CANCELED or +// * FAILED. +// * +// * @return true if the job has the status CANCELED or FAILED, +// * false otherwise +// */ +// private boolean jobHasFailedOrCanceledStatus() { +// +// final Iterator it = new ExecutionGraphIterator(this, true); +// +// while (it.hasNext()) { +// +// final ExecutionState state = it.next().getExecutionState(); +// +// if (state != ExecutionState.CANCELED && state != ExecutionState.FAILED && state != ExecutionState.FINISHED) { +// return false; +// } +// } +// +// return true; +// } +// +// private static InternalJobStatus determineNewJobStatus(final ExecutionGraph eg, +// final ExecutionState latestStateChange) { +// +// final InternalJobStatus currentJobStatus = eg.getJobStatus(); +// +// switch (currentJobStatus) { +// case CREATED: +// if (eg.jobHasScheduledStatus()) { +// return InternalJobStatus.SCHEDULED; +// } else if (latestStateChange == ExecutionState.CANCELED) { +// if (eg.jobHasFailedOrCanceledStatus()) { +// return InternalJobStatus.CANCELED; +// } +// }else if(latestStateChange == ExecutionState.FAILED){ +// return InternalJobStatus.FAILING; +// } +// break; +// case SCHEDULED: +// if (latestStateChange == ExecutionState.RUNNING) { +// return InternalJobStatus.RUNNING; +// } else if (latestStateChange == ExecutionState.CANCELED) { +// if (eg.jobHasFailedOrCanceledStatus()) { +// return InternalJobStatus.CANCELED; +// } +// }else if(latestStateChange == ExecutionState.FAILED){ +// return InternalJobStatus.FAILING; +// } +// break; +// case RUNNING: +// if (latestStateChange == ExecutionState.CANCELED) { +// return InternalJobStatus.CANCELING; +// } +// if (latestStateChange == ExecutionState.FAILED) { +// +// final Iterator it = new ExecutionGraphIterator(eg, true); +// while (it.hasNext()) { +// +// final ExecutionVertex vertex = it.next(); +// if (vertex.getExecutionState() == ExecutionState.FAILED) { +// return InternalJobStatus.FAILING; +// } +// } +// } +// if (eg.jobHasFinishedStatus()) { +// return InternalJobStatus.FINISHED; +// } +// break; +// case FAILING: +// if (eg.jobHasFailedOrCanceledStatus()) { +// return InternalJobStatus.FAILED; +// } +// break; +// case FAILED: +// LOG.error("Received update of execute state in job status FAILED"); +// break; +// case CANCELING: +// if (eg.jobHasFailedOrCanceledStatus()) { +// return InternalJobStatus.CANCELED; +// } +// break; +// case CANCELED: +// LOG.error("Received update of execute state in job status CANCELED: " + eg.getJobID()); +// break; +// case FINISHED: +// LOG.error("Received update of execute state in job status FINISHED: " + eg.getJobID() + " " +// + StringUtils.stringifyException(new Throwable())); +// break; +// } +// +// return currentJobStatus; +// } +// +// /** +// * Returns the current status of the job +// * represented by this execution graph. +// * +// * @return the current status of the job +// */ +// public InternalJobStatus getJobStatus() { +// +// return this.jobStatus.get(); +// } +// +// +// @Override +// public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID, +// final ExecutionState newExecutionState, String optionalMessage) { +// +// // Do not use the parameter newExecutionState here as it may already be out-dated +// +// final ExecutionVertex vertex = getVertexByID(vertexID); +// if (vertex == null) { +// LOG.error("Cannot find execution vertex with the ID " + vertexID); +// return; +// } +// +// final ExecutionState actualExecutionState = vertex.getExecutionState(); +// +// final InternalJobStatus newJobStatus = determineNewJobStatus(this, actualExecutionState); +// +// if (actualExecutionState == ExecutionState.FINISHED) { +// // It is worth checking if the current stage has complete +// if (this.isCurrentStageCompleted()) { +// // Increase current execution stage +// ++this.indexToCurrentExecutionStage; +// +// if (this.indexToCurrentExecutionStage < this.stages.size()) { +// final Iterator it = this.executionStageListeners.iterator(); +// final ExecutionStage nextExecutionStage = getCurrentExecutionStage(); +// while (it.hasNext()) { +// it.next().nextExecutionStageEntered(jobID, nextExecutionStage); +// } +// } +// } +// } +// +// updateJobStatus(newJobStatus, optionalMessage); +// } +// +// /** +// * Updates the job status to given status and triggers the execution of the {@link JobStatusListener} objects. +// * +// * @param newJobStatus +// * the new job status +// * @param optionalMessage +// * an optional message providing details on the reasons for the state change +// */ +// public void updateJobStatus(final InternalJobStatus newJobStatus, String optionalMessage) { +// +// // Check if the new job status equals the old one +// if (this.jobStatus.getAndSet(newJobStatus) == newJobStatus) { +// return; +// } +// +// // The task caused the entire job to fail, save the error description +// if (newJobStatus == InternalJobStatus.FAILING) { +// this.errorDescription = optionalMessage; +// } +// +// // If this is the final failure state change, reuse the saved error description +// if (newJobStatus == InternalJobStatus.FAILED) { +// optionalMessage = this.errorDescription; +// } +// +// final Iterator it = this.jobStatusListeners.iterator(); +// while (it.hasNext()) { +// it.next().jobStatusHasChanged(this, newJobStatus, optionalMessage); +// } +// } +// +// /** +// * Registers a new {@link JobStatusListener} object with this execution graph. +// * After being registered the object will receive notifications about changes +// * of the job status. It is not possible to register the same listener object +// * twice. +// * +// * @param jobStatusListener +// * the listener object to register +// */ +// public void registerJobStatusListener(final JobStatusListener jobStatusListener) { +// +// if (jobStatusListener == null) { +// throw new IllegalArgumentException("Argument jobStatusListener must not be null"); +// } +// +// this.jobStatusListeners.addIfAbsent(jobStatusListener); +// } +// +// /** +// * Unregisters the given {@link JobStatusListener} object. After having called this +// * method, the object will no longer receive notifications about changes of the job +// * status. +// * +// * @param jobStatusListener +// * the listener object to unregister +// */ +// public void unregisterJobStatusListener(final JobStatusListener jobStatusListener) { +// +// if (jobStatusListener == null) { +// throw new IllegalArgumentException("Argument jobStatusListener must not be null"); +// } +// +// this.jobStatusListeners.remove(jobStatusListener); +// } +// +// /** +// * Registers a new {@link ExecutionStageListener} object with this execution graph. After being registered the +// * object will receive a notification whenever the job has entered its next execution stage. Note that a +// * notification is not sent when the job has entered its initial execution stage. +// * +// * @param executionStageListener +// * the listener object to register +// */ +// public void registerExecutionStageListener(final ExecutionStageListener executionStageListener) { +// +// if (executionStageListener == null) { +// throw new IllegalArgumentException("Argument executionStageListener must not be null"); +// } +// +// this.executionStageListeners.addIfAbsent(executionStageListener); +// } +// +// /** +// * Unregisters the given {@link ExecutionStageListener} object. After having called this method, the object will no +// * longer receiver notifications about the execution stage progress. +// * +// * @param executionStageListener +// * the listener object to unregister +// */ +// public void unregisterExecutionStageListener(final ExecutionStageListener executionStageListener) { +// +// if (executionStageListener == null) { +// throw new IllegalArgumentException("Argument executionStageListener must not be null"); +// } +// +// this.executionStageListeners.remove(executionStageListener); +// } +// +// /** +// * Returns the name of the original job graph. +// * +// * @return the name of the original job graph, possibly null +// */ +// public String getJobName() { +// return this.jobName; +// } +// +// private void calculateConnectionIDs() { +// final Set alreadyVisited = new HashSet(); +// final ExecutionStage lastStage = getStage(getNumberOfStages() - 1); +// +// for (int i = 0; i < lastStage.getNumberOfStageMembers(); ++i) { +// +// final ExecutionGroupVertex groupVertex = lastStage.getStageMember(i); +// +// int currentConnectionID = 0; +// +// if (groupVertex.isOutputVertex()) { +// currentConnectionID = groupVertex.calculateConnectionID(currentConnectionID, alreadyVisited); +// } +// } +// } +// +// /** +// * Retrieves the number of required slots to run this execution graph +// * @return +// */ +// public int getRequiredSlots(){ +// int maxRequiredSlots = 0; +// +// final Iterator stageIterator = this.stages.iterator(); +// +// while(stageIterator.hasNext()){ +// final ExecutionStage stage = stageIterator.next(); +// +// int requiredSlots = stage.getRequiredSlots(); +// +// if(requiredSlots > maxRequiredSlots){ +// maxRequiredSlots = requiredSlots; +// } +// } +// +// return maxRequiredSlots; +// } +//} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.old similarity index 59% rename from flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.old index 26dd19edb190c67a4d86d2407bcabc8a90fde7f8..beec8497cb6b9048a5aa0fd3dc47ca19d27c48bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.old @@ -73,8 +73,101 @@ public class ExecutionVertex { // -------------------------------------------------------------------------------------------- - public ExecutionVertex(ExecutionJobVertex jobVertex, int subTaskIndex, IntermediateResult[] producedDataSets) { - this(jobVertex, subTaskIndex, producedDataSets, System.currentTimeMillis()); + /** + * The ID of the vertex. + */ + private final ExecutionVertexID vertexID; + + /** + * The group vertex this vertex belongs to. + */ + private final ExecutionGroupVertex groupVertex; + + /** + * The execution graph is vertex belongs to. + */ + private final ExecutionGraph executionGraph; + + /** + * The allocated resources assigned to this vertex. + */ + private final AtomicReference allocatedResource = new AtomicReference(null); + + /** + * The allocation ID identifying the allocated resources used by this vertex + * within the instance. + */ + private volatile AllocationID allocationID = null; + + /** + * A list of {@link VertexAssignmentListener} objects to be notified about changes in the instance assignment. + */ + private final CopyOnWriteArrayList vertexAssignmentListeners = new CopyOnWriteArrayList(); + + private final CopyOnWriteArrayList vertexAssignmentListenerActors = new + CopyOnWriteArrayList(); + + /** + * A map of {@link ExecutionListener} objects to be notified about the state changes of a vertex. + */ + private final ConcurrentMap executionListeners = new ConcurrentSkipListMap(); + + private final CopyOnWriteArrayList executionListenerActors = new CopyOnWriteArrayList(); + + /** + * The current execution state of the task represented by this vertex + */ + private final AtomicEnum executionState = new AtomicEnum(ExecutionState.CREATED); + + /** + * The output gates attached to this vertex. + */ + private final ExecutionGate[] outputGates; + + /** + * The input gates attached to his vertex. + */ + private final ExecutionGate[] inputGates; + + /** + * The index of this vertex in the vertex group. + */ + private volatile int indexInVertexGroup = 0; + + /** + * Stores the number of times the vertex may be still be started before the corresponding task is considered to be + * failed. + */ + private final AtomicInteger retriesLeft; + + + /** + * The execution pipeline this vertex is part of. + */ + private final AtomicReference executionPipeline = new AtomicReference(null); + + /** + * Flag to indicate whether the vertex has been requested to cancel while in state STARTING + */ + private final AtomicBoolean cancelRequested = new AtomicBoolean(false); + + /** + * Create a new execution vertex and instantiates its environment. + * + * @param executionGraph + * the execution graph the new vertex belongs to + * @param groupVertex + * the group vertex the new vertex belongs to + * @param numberOfOutputGates + * the number of output gates attached to this vertex + * @param numberOfInputGates + * the number of input gates attached to this vertex + */ + public ExecutionVertex(final ExecutionGraph executionGraph, final ExecutionGroupVertex groupVertex, + final int numberOfOutputGates, final int numberOfInputGates) { + this(new ExecutionVertexID(), executionGraph, groupVertex, numberOfOutputGates, numberOfInputGates); + + this.groupVertex.addInitialSubtask(this); } public ExecutionVertex(ExecutionJobVertex jobVertex, int subTaskIndex, IntermediateResult[] producedDataSets, long createTimestamp) { @@ -122,22 +215,19 @@ public class ExecutionVertex { public String getTaskName() { return this.jobVertex.getJobVertex().getName(); } - - public int getTotalNumberOfParallelSubtasks() { - return this.jobVertex.getParallelism(); - } - - public int getParallelSubtaskIndex() { - return this.subTaskIndex; - } - - public int getNumberOfInputs() { - return this.inputEdges.length; - } - - public ExecutionEdge[] getInputEdges(int input) { - if (input < 0 || input >= this.inputEdges.length) { - throw new IllegalArgumentException(String.format("Input %d is out of range [0..%d)", input, this.inputEdges.length)); + + /** + * Inserts the input gate at the given position. + * + * @param pos + * the position to insert the input gate + * @param inputGate + * the input gate to be inserted + */ + void insertInputGate(final int pos, final ExecutionGate inputGate) { + + if (this.inputGates[pos] != null) { + throw new IllegalStateException("Input gate at position " + pos + " is not null"); } return inputEdges[input]; } @@ -165,9 +255,62 @@ public class ExecutionVertex { public AllocatedSlot getCurrentAssignedResource() { return currentExecution.getAssignedResource(); } - - public ExecutionGraph getExecutionGraph() { - return this.jobVertex.getGraph(); + + /** + * Updates the vertex's current execution state. + * + * @param newExecutionState + * the new execution state + * @param optionalMessage + * an optional message related to the state change + */ + public ExecutionState updateExecutionState(ExecutionState newExecutionState, final String optionalMessage) { + + if (newExecutionState == null) { + throw new IllegalArgumentException("Argument newExecutionState must not be null"); + } + + final ExecutionState currentExecutionState = this.executionState.get(); + if (currentExecutionState == ExecutionState.CANCELING) { + + // If we are in CANCELING, ignore state changes to FINISHING + if (newExecutionState == ExecutionState.FINISHING) { + return currentExecutionState; + } + + // Rewrite FINISHED to CANCELED if the task has been marked to be canceled + if (newExecutionState == ExecutionState.FINISHED) { + LOG.info("Received transition from CANCELING to FINISHED for vertex " + toString() + + ", converting it to CANCELED"); + newExecutionState = ExecutionState.CANCELED; + } + } + + // Check and save the new execution state + final ExecutionState previousState = this.executionState.getAndSet(newExecutionState); + if (previousState == newExecutionState) { + return previousState; + } + + // Check the transition + ExecutionStateTransition.checkTransition(true, toString(), previousState, newExecutionState); + + // Notify the listener objects + final Iterator it = this.executionListeners.values().iterator(); + while (it.hasNext()) { + it.next().executionStateChanged(this.executionGraph.getJobID(), this.vertexID, newExecutionState, + optionalMessage); + } + + for(ActorRef actor: executionListenerActors){ + actor.tell(new ExecutionGraphMessages.ExecutionStateChanged(this.executionGraph.getJobID(), + this.vertexID, newExecutionState, optionalMessage), ActorRef.noSender()); + } + + // The vertex was requested to be canceled by another thread + checkCancelRequestedFlag(); + + return previousState; } // -------------------------------------------------------------------------------------------- @@ -206,6 +349,26 @@ public class ExecutionVertex { ee.getSource().addConsumer(ee, consumerNumber); graph.registerExecutionEdge(ee); } + + // Check the transition + ExecutionStateTransition.checkTransition(true, toString(), expected, update); + + // Notify the listener objects + final Iterator it = this.executionListeners.values().iterator(); + while (it.hasNext()) { + it.next().executionStateChanged(this.executionGraph.getJobID(), this.vertexID, update, + null); + } + + for(ActorRef actor: executionListenerActors){ + actor.tell(new ExecutionGraphMessages.ExecutionStateChanged(this.executionGraph.getJobID(), + this.vertexID, update, null), ActorRef.noSender()); + } + + // Check if the vertex was requested to be canceled by another thread + checkCancelRequestedFlag(); + + return true; } private ExecutionEdge[] connectAllToAll(IntermediateResultPartition[] sourcePartitions, int inputNumber) { @@ -218,14 +381,17 @@ public class ExecutionVertex { return edges; } - - private ExecutionEdge[] connectPointwise(IntermediateResultPartition[] sourcePartitions, int inputNumber) { - final int numSources = sourcePartitions.length; - final int parallelism = getTotalNumberOfParallelSubtasks(); - - // simple case same number of sources as targets - if (numSources == parallelism) { - return new ExecutionEdge[] { new ExecutionEdge(sourcePartitions[subTaskIndex], this, inputNumber) }; + + /** + * Assigns the execution vertex with an {@link org.apache.flink.runtime.instance.AllocatedResource}. + * + * @param allocatedResource + * the resources which are supposed to be allocated to this vertex + */ + public void setAllocatedResource(final AllocatedResource allocatedResource) { + + if (allocatedResource == null) { + throw new IllegalArgumentException("Argument allocatedResource must not be null"); } else if (numSources < parallelism) { @@ -274,6 +440,11 @@ public class ExecutionVertex { return edges; } } + + for(ActorRef actor: vertexAssignmentListenerActors){ + actor.tell(new ExecutionGraphMessages.VertexAssignmentChanged(this.getExecutionGraph().getJobID(),vertexID, + allocatedResource), ActorRef.noSender()); + } } /** @@ -384,12 +555,50 @@ public class ExecutionVertex { void notifyStateTransition(ExecutionAttemptID executionId, ExecutionState newState, Throwable error) { getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, executionId, newState, error); } - - TaskDeploymentDescriptor createDeploymentDescriptor(ExecutionAttemptID executionId, AllocatedSlot slot) { - // create the input gate deployment descriptors - List inputGates = new ArrayList(inputEdges.length); - for (ExecutionEdge[] channels : inputEdges) { - inputGates.add(GateDeploymentDescriptor.fromEdges(channels)); + + /** + * Registers the {@link VertexAssignmentListener} object for this vertex. This object + * will be notified about reassignments of this vertex to another instance. + * + * @param vertexAssignmentListener + * the object to be notified about reassignments of this vertex to another instance + */ + public void registerVertexAssignmentListener(final VertexAssignmentListener vertexAssignmentListener) { + + this.vertexAssignmentListeners.addIfAbsent(vertexAssignmentListener); + } + + public void registerVertexAssignmentListener(final ActorRef vertexAssignmentListener){ + this.vertexAssignmentListenerActors.addIfAbsent(vertexAssignmentListener); + } + + /** + * Unregisters the {@link VertexAssignmentListener} object for this vertex. This object + * will no longer be notified about reassignments of this vertex to another instance. + * + * @param vertexAssignmentListener + * the listener to be unregistered + */ + public void unregisterVertexAssignmentListener(final VertexAssignmentListener vertexAssignmentListener) { + + this.vertexAssignmentListeners.remove(vertexAssignmentListener); + } + + + /** + * Registers the {@link ExecutionListener} object for this vertex. This object + * will be notified about particular events during the vertex's lifetime. + * + * @param executionListener + * the object to be notified about particular events during the vertex's lifetime + */ + public void registerExecutionListener(final ExecutionListener executionListener) { + + final Integer priority = Integer.valueOf(executionListener.getPriority()); + + if (priority.intValue() < 0) { + LOG.error("Priority for execution listener " + executionListener.getClass() + " must be non-negative."); + return; } // create the output gate deployment descriptors @@ -407,14 +616,21 @@ public class ExecutionVertex { getExecutionGraph().getJobConfiguration(), jobVertex.getJobVertex().getConfiguration(), jobVertex.getJobVertex().getInvokableClassName(), outputGates, inputGates, jarFiles, slot.getSlotNumber()); } - - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - public void execute(Runnable action) { - this.jobVertex.execute(action); + + public void registerExecutionListener(final ActorRef executionListener){ + this.executionListenerActors.addIfAbsent(executionListener); + } + + /** + * Unregisters the {@link ExecutionListener} object for this vertex. This object + * will no longer be notified about particular events during the vertex's lifetime. + * + * @param executionListener + * the object to be unregistered + */ + public void unregisterExecutionListener(final ExecutionListener executionListener) { + + this.executionListeners.remove(Integer.valueOf(executionListener.getPriority())); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java index f69973ffd3767f909859555839b1959640f29676..92f7414d8a0a903fbf468a5e0c01e0ff91d98313 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java @@ -18,16 +18,12 @@ package org.apache.flink.runtime.instance; -import java.io.IOException; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; +import java.io.Serializable; /** * A hardware description describes the resources available to a task manager. */ -public final class HardwareDescription implements IOReadableWritable, java.io.Serializable { +public final class HardwareDescription implements Serializable { private static final long serialVersionUID = 3380016608300325361L; @@ -100,26 +96,6 @@ public final class HardwareDescription implements IOReadableWritable, java.io.Se return this.sizeOfManagedMemory; } - // -------------------------------------------------------------------------------------------- - // Serialization - // -------------------------------------------------------------------------------------------- - - @Override - public void write(DataOutputView out) throws IOException { - out.writeInt(this.numberOfCPUCores); - out.writeLong(this.sizeOfPhysicalMemory); - out.writeLong(this.sizeOfJvmHeap); - out.writeLong(this.sizeOfManagedMemory); - } - - @Override - public void read(DataInputView in) throws IOException { - this.numberOfCPUCores = in.readInt(); - this.sizeOfPhysicalMemory = in.readLong(); - this.sizeOfJvmHeap = in.readLong(); - this.sizeOfManagedMemory = in.readLong(); - } - // -------------------------------------------------------------------------------------------- // Utils // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java index 89453bf3bfe6f5e04cc43060289a4692674cae1d..9a0fddb6889f069e86282adf6baf693df9b21396 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.instance; import java.io.IOException; -import java.net.InetSocketAddress; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.HashSet; @@ -27,28 +26,35 @@ import java.util.List; import java.util.Queue; import java.util.Set; -import org.apache.flink.runtime.ipc.RPC; +import akka.actor.ActorRef; +import akka.dispatch.ExecutionContexts; +import akka.dispatch.Futures; +import akka.dispatch.Mapper; +import akka.pattern.Patterns; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailabilityListener; -import org.apache.flink.runtime.net.NetUtils; -import org.apache.flink.runtime.protocols.TaskOperationProtocol; -import org.eclipse.jetty.util.log.Log; +import org.apache.flink.runtime.messages.TaskManagerMessages; +import org.apache.flink.runtime.taskmanager.TaskOperationResult; +import scala.concurrent.Await; +import scala.concurrent.Future; /** - * An instance represents a resource a {@link org.apache.flink.runtime.taskmanager.TaskManager} runs on. + * An taskManager represents a resource a {@link org.apache.flink.runtime.taskmanager.TaskManager} runs on. */ public class Instance { /** The lock on which to synchronize allocations and failure state changes */ private final Object instanceLock = new Object(); - /** The connection info to connect to the task manager represented by this instance. */ - private final InstanceConnectionInfo instanceConnectionInfo; + /** The actor ref to the task manager represented by this taskManager. */ + private final ActorRef taskManager; /** A description of the resources of the task manager */ private final HardwareDescription resources; - /** The ID identifying the instance. */ + /** The ID identifying the taskManager. */ private final InstanceID instanceId; /** The number of task slots available on the node */ @@ -57,19 +63,15 @@ public class Instance { /** A list of available slot positions */ private final Queue availableSlots; - /** Allocated slots on this instance */ + /** Allocated slots on this taskManager */ private final Set allocatedSlots = new HashSet(); /** A listener to be notified upon new slot availability */ private SlotAvailabilityListener slotAvailabilityListener; - - /** The RPC proxy to send calls to the task manager represented by this instance */ - private volatile TaskOperationProtocol taskManager; - /** - * Time when last heat beat has been received from the task manager running on this instance. + * Time when last heat beat has been received from the task manager running on this taskManager. */ private volatile long lastReceivedHeartBeat = System.currentTimeMillis(); @@ -78,15 +80,15 @@ public class Instance { // -------------------------------------------------------------------------------------------- /** - * Constructs an abstract instance object. + * Constructs an abstract taskManager object. * - * @param instanceConnectionInfo The connection info under which to reach the TaskManager instance. - * @param id The id under which the instance is registered. + * @param taskManager The actor reference of the represented task manager. + * @param id The id under which the taskManager is registered. * @param resources The resources available on the machine. - * @param numberOfSlots The number of task slots offered by this instance. + * @param numberOfSlots The number of task slots offered by this taskManager. */ - public Instance(InstanceConnectionInfo instanceConnectionInfo, InstanceID id, HardwareDescription resources, int numberOfSlots) { - this.instanceConnectionInfo = instanceConnectionInfo; + public Instance(ActorRef taskManager, InstanceID id, HardwareDescription resources, int numberOfSlots) { + this.taskManager = taskManager; this.instanceId = id; this.resources = resources; this.numberOfSlots = numberOfSlots; @@ -113,15 +115,6 @@ public class Instance { return numberOfSlots; } - /** - * Returns the instance's connection information object. - * - * @return the instance's connection information object - */ - public InstanceConnectionInfo getInstanceConnectionInfo() { - return this.instanceConnectionInfo; - } - // -------------------------------------------------------------------------------------------- // Life and Death // -------------------------------------------------------------------------------------------- @@ -173,49 +166,72 @@ public class Instance { allocatedSlots.clear(); availableSlots.clear(); } - - destroyTaskManagerProxy(); } - // -------------------------------------------------------------------------------------------- - // Connection to the TaskManager - // -------------------------------------------------------------------------------------------- - - public TaskOperationProtocol getTaskManagerProxy() throws IOException { - if (isDead) { - throw new IOException("Instance has died"); + public void checkLibraryAvailability(final JobID jobID) throws IOException { + final String[] requiredLibraries = LibraryCacheManager.getRequiredJarFiles(jobID); + + if (requiredLibraries == null) { + throw new IOException("No entry of required libraries for job " + jobID); } + + LibraryCacheProfileRequest request = new LibraryCacheProfileRequest(); + request.setRequiredLibraries(requiredLibraries); + + Future futureResponse = Patterns.ask(taskManager, new TaskManagerMessages.RequestLibraryCacheProfile + (request), AkkaUtils.FUTURE_TIMEOUT()); - TaskOperationProtocol tm = this.taskManager; - - if (tm == null) { - synchronized (this) { - if (this.taskManager == null) { - this.taskManager = RPC.getProxy(TaskOperationProtocol.class, - new InetSocketAddress(getInstanceConnectionInfo().address(), - getInstanceConnectionInfo().ipcPort()), NetUtils.getSocketFactory()); + + Future> updateFuture = futureResponse.flatMap(new Mapper>>() { + public Future> apply(final Object o) { + LibraryCacheProfileResponse response = (LibraryCacheProfileResponse) o; + + List> futureAcks = new ArrayList>(); + + for (int i = 0; i < requiredLibraries.length; i++) { + if (!response.isCached(i)) { + LibraryCacheUpdate update = new LibraryCacheUpdate(requiredLibraries[i]); + Future future = Patterns.ask(taskManager, update, AkkaUtils.FUTURE_TIMEOUT()); + futureAcks.add(future); + } } - tm = this.taskManager; + + return Futures.sequence(futureAcks, ExecutionContexts.global()); } + }, ExecutionContexts.global()); + + try { + Await.result(updateFuture, AkkaUtils.AWAIT_DURATION()); + }catch(IOException ioe){ + throw ioe; + }catch(Exception e){ + throw new RuntimeException("Encountered exception while updating library cache.", e); } - - return tm; + } - /** Destroys and removes the RPC stub object for this instance's task manager. */ - private void destroyTaskManagerProxy() { - synchronized (this) { - if (this.taskManager != null) { - try { - RPC.stopProxy(this.taskManager); - } catch (Throwable t) { - Log.debug("Error shutting down RPC proxy.", t); - } - } + public TaskOperationResult submitTask(TaskDeploymentDescriptor tdd) throws IOException{ + Future futureResponse = Patterns.ask(taskManager, new TaskManagerMessages.SubmitTask(tdd), + AkkaUtils.FUTURE_TIMEOUT()); + try{ + return (TaskOperationResult) Await.result(futureResponse, AkkaUtils.AWAIT_DURATION()); + }catch(IOException ioe){ + throw ioe; + }catch(Exception e){ + throw new RuntimeException("Caught exception while submitting task.", e); } } - + public TaskOperationResult cancelTask(JobVertexID jobVertexID, int subtaskIndex) throws IOException{ + Future futureResponse = Patterns.ask(taskManager, new TaskManagerMessages.CancelTask(jobVertexID, + subtaskIndex), AkkaUtils.FUTURE_TIMEOUT()); + try{ + return (TaskOperationResult) Await.result(futureResponse, AkkaUtils.AWAIT_DURATION()); + }catch(IOException ioe){ + throw ioe; + }catch(Exception e){ + throw new RuntimeException("Caught exception while cancelling task.", e); + } // -------------------------------------------------------------------------------------------- // Heartbeats @@ -243,7 +259,7 @@ public class Instance { * * @param now The timestamp representing the current time. * @param cleanUpInterval The maximum time (in msecs) that the last heartbeat may lie in the past. - * @return True, if this instance is considered alive, false otherwise. + * @return True, if this taskManager is considered alive, false otherwise. */ public boolean isStillAlive(long now, long cleanUpInterval) { return this.lastReceivedHeartBeat + cleanUpInterval > now; @@ -275,9 +291,9 @@ public class Instance { } public boolean returnAllocatedSlot(AllocatedSlot slot) { - // the slot needs to be in the returned to instance state + // the slot needs to be in the returned to taskManager state if (slot == null || slot.getInstance() != this) { - throw new IllegalArgumentException("Slot is null or belongs to the wrong instance."); + throw new IllegalArgumentException("Slot is null or belongs to the wrong taskManager."); } if (slot.isAlive()) { throw new IllegalArgumentException("Slot is still alive"); @@ -298,7 +314,7 @@ public class Instance { return true; } else { - throw new IllegalArgumentException("Slot was not allocated from the instance."); + throw new IllegalArgumentException("Slot was not allocated from the taskManager."); } } } else { @@ -317,6 +333,14 @@ public class Instance { allocatedSlots.clear(); } } + + public ActorRef getTaskManager() { + return taskManager; + } + + public String getPath(){ + return taskManager.path().toString(); + } public int getNumberOfAvailableSlots() { return this.availableSlots.size(); @@ -356,6 +380,6 @@ public class Instance { @Override public String toString() { - return instanceId + " @" + this.instanceConnectionInfo + ' ' + numberOfSlots + " slots"; + return instanceId + " @" + taskManager.path() + ' ' + numberOfSlots + " slots"; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java index f779048bd1cdcb26031b9ca191b0e38f093e78c0..eac36a06af826b9d75face642d57ffe6555ff708 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java @@ -28,6 +28,7 @@ import java.util.Set; import java.util.Timer; import java.util.TimerTask; +import akka.actor.ActorRef; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.GlobalConfiguration; import org.slf4j.Logger; @@ -51,10 +52,10 @@ public class InstanceManager { private final Map registeredHostsById; /** Set of hosts known to run a task manager that are thus able to execute tasks (by connection). */ - private final Map registeredHostsByConnection; + private final Map registeredHostsByConnection; /** Set of hosts that were present once and have died */ - private final Set deadHosts; + private final Set deadHosts; /** Listeners that want to be notified about availability and disappearance of instances */ private final List instanceListeners = new ArrayList(); @@ -92,8 +93,8 @@ public class InstanceManager { } this.registeredHostsById = new HashMap(); - this.registeredHostsByConnection = new HashMap(); - this.deadHosts = new HashSet(); + this.registeredHostsByConnection = new HashMap(); + this.deadHosts = new HashSet(); this.heartbeatTimeout = heartbeatTimeout; new Timer(true).schedule(cleanupStaleMachines, cleanupInterval, cleanupInterval); @@ -159,22 +160,23 @@ public class InstanceManager { } } - public InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription resources, int numberOfSlots){ + public InstanceID registerTaskManager(ActorRef taskManager, HardwareDescription resources, + int numberOfSlots){ synchronized(this.lock){ if (this.shutdown) { throw new IllegalStateException("InstanceManager is shut down."); } - Instance prior = registeredHostsByConnection.get(instanceConnectionInfo); + Instance prior = registeredHostsByConnection.get(taskManager); if (prior != null) { - LOG.error("Registration attempt from TaskManager with connection info " + instanceConnectionInfo + + LOG.error("Registration attempt from TaskManager at " + taskManager.path() + ". This connection is already registered under ID " + prior.getId()); return null; } - boolean wasDead = this.deadHosts.remove(instanceConnectionInfo); + boolean wasDead = this.deadHosts.remove(taskManager); if (wasDead) { - LOG.info("Registering TaskManager with connection info " + instanceConnectionInfo + + LOG.info("Registering TaskManager at " + taskManager.path() + " which was marked as dead earlier because of a heart-beat timeout."); } @@ -184,16 +186,16 @@ public class InstanceManager { } while (registeredHostsById.containsKey(id)); - Instance host = new Instance(instanceConnectionInfo, id, resources, numberOfSlots); + Instance host = new Instance(taskManager, id, resources, numberOfSlots); registeredHostsById.put(id, host); - registeredHostsByConnection.put(instanceConnectionInfo, host); + registeredHostsByConnection.put(taskManager, host); totalNumberOfAliveTaskSlots += numberOfSlots; if (LOG.isInfoEnabled()) { LOG.info(String.format("Registered TaskManager at %s as %s. Current number of registered hosts is %d.", - instanceConnectionInfo, id, registeredHostsById.size())); + taskManager.path(), id, registeredHostsById.size())); } host.reportHeartBeat(); @@ -284,17 +286,17 @@ public class InstanceManager { // remove from the living entries.remove(); - registeredHostsByConnection.remove(host.getInstanceConnectionInfo()); + registeredHostsByConnection.remove(host.getTaskManager()); // add to the dead - deadHosts.add(host.getInstanceConnectionInfo()); + deadHosts.add(host.getTaskManager()); host.markDead(); totalNumberOfAliveTaskSlots -= host.getTotalNumberOfSlots(); LOG.info(String.format("TaskManager %s at %s did not report a heartbeat for %d msecs - marking as dead. Current number of registered hosts is %d.", - host.getId(), host.getInstanceConnectionInfo(), heartbeatTimeout, registeredHostsById.size())); + host.getId(), host.getPath(), heartbeatTimeout, registeredHostsById.size())); // report to all listeners notifyDeadInstance(host); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java deleted file mode 100644 index 34f6100d6b547eb6897706b6a6d10dde0b385b7c..0000000000000000000000000000000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java +++ /dev/null @@ -1,88 +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. - */ - -package org.apache.flink.runtime.instance; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.runtime.ExecutionMode; -import org.apache.flink.runtime.taskmanager.TaskManager2; -import org.slf4j.LoggerFactory; - -/** - * A variant of the {@link InstanceManager} that internally spawn task managers as instances, rather than waiting for external - * TaskManagers to register. - */ -public class LocalInstanceManager extends InstanceManager { - - private final List taskManagers = new ArrayList(); - - - public LocalInstanceManager(int numTaskManagers) throws Exception { - ExecutionMode execMode = numTaskManagers == 1 ? ExecutionMode.LOCAL : ExecutionMode.CLUSTER; - - final int ipcPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, -1); - final int dataPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, -1); - - for (int i = 0; i < numTaskManagers; i++) { - - // configure ports, if necessary - if (ipcPort > 0 || dataPort > 0) { - Configuration tm = new Configuration(); - if (ipcPort > 0) { - tm.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, ipcPort + i); - } - if (dataPort > 0) { - tm.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort + i); - } - - GlobalConfiguration.includeConfiguration(tm); - } - - taskManagers.add(TaskManager.createTaskManager(execMode)); - } - } - - @Override - public void shutdown() { - try { - for (TaskManager taskManager: taskManagers){ - try { - taskManager.shutdown(); - } - catch (Throwable t) { - // log and continue in any case - // we initialize the log lazily, because this is the only place we log - // and most likely we never log. - LoggerFactory.getLogger(LocalInstanceManager.class).error("Error shutting down local embedded TaskManager.", t); - } - } - } finally { - this.taskManagers.clear(); - super.shutdown(); - } - } - - public TaskManager[] getTaskManagers() { - return (TaskManager[]) this.taskManagers.toArray(new TaskManager[this.taskManagers.size()]); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/serialization/DataInputDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/serialization/DataInputDeserializer.java index c8adaea64692106df62db1c715e71b84aaec6913..66db4a251a97c7c1521832a10b3a1c8c7aae4fc6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/serialization/DataInputDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/serialization/DataInputDeserializer.java @@ -77,7 +77,7 @@ public class DataInputDeserializer implements DataInputView { this.buffer = buffer; this.position = start; - this.end = start * len; + this.end = start + len; } // ---------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.old similarity index 98% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.old index 7fd089f8a7c7177b50a3af4a3d67edd4e46e2fa7..a61ce46efdf2e56321567bd5625a384bf13f588b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.old @@ -52,7 +52,7 @@ import org.slf4j.LoggerFactory; * for being fetched by a client. The collected events have an expiration time. In a configurable interval * the event collector removes all intervals which are older than the interval. */ -public final class EventCollector extends TimerTask implements ProfilingListener { +public final class EventCollector2 extends TimerTask implements ProfilingListener { private static final Logger LOG = LoggerFactory.getLogger(EventCollector.class); @@ -91,7 +91,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener final VertexEvent vertexEvent = new VertexEvent(timestamp, vertexId, taskName, totalNumberOfSubtasks, subtask, executionId, newExecutionState, optionalMessage); - this.eventCollector.addEvent(jobID, vertexEvent); + this.eventCollector2.addEvent(jobID, vertexEvent); final ExecutionStateChangeEvent executionStateChangeEvent = new ExecutionStateChangeEvent(timestamp, vertexId, subtask, executionId, newExecutionState); @@ -126,7 +126,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener /** * Constructs a new job status listener wrapper. * - * @param eventCollector + * @param eventCollector2 * the event collector to forward the events to * @param jobName * the name of the job @@ -194,7 +194,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener * @param clientQueryInterval * the interval with which clients query for events */ - public EventCollector(final int clientQueryInterval) { + public EventCollector2(final int clientQueryInterval) { this.timerTaskInterval = clientQueryInterval * 1000L * 2L; // Double the interval, clients will take care of // duplicate notifications diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java index 0cd08f5aa50b998178be1596052adc6503ce1cfe..73c33747d7110e4be41fd3d90dc694e4f0c301a4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java @@ -82,8 +82,8 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.accumulators.AccumulatorManager; import org.apache.flink.runtime.jobmanager.archive.ArchiveListener; -import org.apache.flink.runtime.jobmanager.archive.MemoryArchivist; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; +import org.apache.flink.runtime.jobmanager.archive.MemoryArchivist2; +import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler; import org.apache.flink.runtime.jobmanager.web.WebInfoServer; import org.apache.flink.runtime.protocols.AccumulatorProtocol; import org.apache.flink.runtime.protocols.ChannelLookupProtocol; @@ -187,7 +187,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide ConfigConstants.DEFAULT_JOB_MANAGER_DEAD_TASKMANAGER_TIMEOUT); // Load the job progress collector - this.eventCollector = new EventCollector(this.recommendedClientPollingInterval); + this.eventCollector2 = new EventCollector2(this.recommendedClientPollingInterval); this.libraryCacheManager = new BlobLibraryCacheManager(new BlobServer(), GlobalConfiguration.getConfiguration()); @@ -196,8 +196,8 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide int archived_items = GlobalConfiguration.getInteger( ConfigConstants.JOB_MANAGER_WEB_ARCHIVE_COUNT, ConfigConstants.DEFAULT_JOB_MANAGER_WEB_ARCHIVE_COUNT); if (archived_items > 0) { - this.archive = new MemoryArchivist(archived_items); - this.eventCollector.registerArchivist(archive); + this.archive = new MemoryArchivist2(archived_items); + this.eventCollector2.registerArchivist(archive); } else { this.archive = null; @@ -286,8 +286,8 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide } // Stop and clean up the job progress collector - if (this.eventCollector != null) { - this.eventCollector.shutdown(); + if (this.eventCollector2 != null) { + this.eventCollector2.shutdown(); } // Finally, shut down the scheduler @@ -541,7 +541,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide } final SerializableArrayList eventList = new SerializableArrayList(); - this.eventCollector.getEventsForJob(jobID, eventList, false); + this.eventCollector2.getEventsForJob(jobID, eventList, false); return new JobProgressResult(ReturnCode.SUCCESS, null, eventList); } @@ -586,11 +586,11 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide final List eventList = new SerializableArrayList(); - if (this.eventCollector == null) { + if (this.eventCollector2 == null) { throw new IOException("No instance of the event collector found"); } - this.eventCollector.getRecentJobs(eventList); + this.eventCollector2.getRecentJobs(eventList); return eventList; } @@ -601,11 +601,11 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide final List eventList = new SerializableArrayList(); - if (this.eventCollector == null) { + if (this.eventCollector2 == null) { throw new IOException("No instance of the event collector found"); } - this.eventCollector.getEventsForJob(jobID, eventList, true); + this.eventCollector2.getEventsForJob(jobID, eventList, true); return eventList; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist2.old similarity index 98% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist2.old index 24a5cf912eef93e308b94fe27fce5ec1c0f7a668..746c90cb78875bbb6f8ec3bea6d491edc2e509b9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist2.old @@ -35,7 +35,7 @@ import org.apache.flink.runtime.jobgraph.JobID; * This class must be thread safe, because it is accessed by the JobManager events and by the * web server concurrently. */ -public class MemoryArchivist implements ArchiveListener { +public class MemoryArchivist2 implements ArchiveListener { /** The global lock */ private final Object lock = new Object(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java index ef5a2462905cf95be32af427cc74b16d3dd29c06..75f09652cc540ad16321d06f1fa1b2345dcce968 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java @@ -61,10 +61,10 @@ public class JobmanagerInfoServlet extends HttpServlet { private static final Logger LOG = LoggerFactory.getLogger(JobmanagerInfoServlet.class); /** Underlying JobManager */ - private final JobManager jobmanager; + private final ActorRef jobmanager; - public JobmanagerInfoServlet(JobManager jobmanager) { + public JobmanagerInfoServlet(ActorRef jobmanager) { this.jobmanager = jobmanager; } @@ -72,46 +72,46 @@ public class JobmanagerInfoServlet extends HttpServlet { @Override protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException { - resp.setStatus(HttpServletResponse.SC_OK); - resp.setContentType("application/json"); - - try { - if("archive".equals(req.getParameter("get"))) { - writeJsonForArchive(resp.getWriter(), jobmanager.getOldJobs()); - } - else if("job".equals(req.getParameter("get"))) { - String jobId = req.getParameter("job"); - writeJsonForArchivedJob(resp.getWriter(), jobmanager.getArchive().getJob(JobID.fromHexString(jobId))); - } - else if("groupvertex".equals(req.getParameter("get"))) { - String jobId = req.getParameter("job"); - String groupvertexId = req.getParameter("groupvertex"); - writeJsonForArchivedJobGroupvertex(resp.getWriter(), jobmanager.getArchive().getJob(JobID.fromHexString(jobId)), JobVertexID.fromHexString(groupvertexId)); - } - else if("taskmanagers".equals(req.getParameter("get"))) { +// resp.setStatus(HttpServletResponse.SC_OK); +// resp.setContentType("application/json"); +// +// try { +// if("archive".equals(req.getParameter("get"))) { +// writeJsonForArchive(resp.getWriter(), jobmanager.getOldJobs()); +// } +// else if("job".equals(req.getParameter("get"))) { +// String jobId = req.getParameter("job"); +// writeJsonForArchivedJob(resp.getWriter(), jobmanager.getArchive().getJob(JobID.fromHexString(jobId))); +// } +// else if("groupvertex".equals(req.getParameter("get"))) { +// String jobId = req.getParameter("job"); +// String groupvertexId = req.getParameter("groupvertex"); +// writeJsonForArchivedJobGroupvertex(resp.getWriter(), jobmanager.getArchive().getJob(JobID.fromHexString(jobId)), JobVertexID.fromHexString(groupvertexId)); +// } +// else if("taskmanagers".equals(req.getParameter("get"))) { resp.getWriter().write("{\"taskmanagers\": " + jobmanager.getNumberOfTaskManagers() +", \"slots\": "+jobmanager.getTotalNumberOfRegisteredSlots()+"}"); - } - else if("cancel".equals(req.getParameter("get"))) { - String jobId = req.getParameter("job"); - jobmanager.cancelJob(JobID.fromHexString(jobId)); - } - else if("updates".equals(req.getParameter("get"))) { - String jobId = req.getParameter("job"); - writeJsonUpdatesForJob(resp.getWriter(), JobID.fromHexString(jobId)); - } else if ("version".equals(req.getParameter("get"))) { - writeJsonForVersion(resp.getWriter()); - } - else{ - writeJsonForJobs(resp.getWriter(), jobmanager.getRecentJobs()); - } - - } catch (Exception e) { - resp.setStatus(HttpServletResponse.SC_BAD_REQUEST); - resp.getWriter().print(e.getMessage()); - if (LOG.isWarnEnabled()) { - LOG.warn(StringUtils.stringifyException(e)); - } - } +// } +// else if("cancel".equals(req.getParameter("get"))) { +// String jobId = req.getParameter("job"); +// jobmanager.cancelJob(JobID.fromHexString(jobId)); +// } +// else if("updates".equals(req.getParameter("get"))) { +// String jobId = req.getParameter("job"); +// writeJsonUpdatesForJob(resp.getWriter(), JobID.fromHexString(jobId)); +// } else if ("version".equals(req.getParameter("get"))) { +// writeJsonForVersion(resp.getWriter()); +// } +// else{ +// writeJsonForJobs(resp.getWriter(), jobmanager.getRecentJobs()); +// } +// +// } catch (Exception e) { +// resp.setStatus(HttpServletResponse.SC_BAD_REQUEST); +// resp.getWriter().print(e.getMessage()); +// if (LOG.isWarnEnabled()) { +// LOG.warn(StringUtils.stringifyException(e)); +// } +// } } /** @@ -149,30 +149,30 @@ public class JobmanagerInfoServlet extends HttpServlet { private void writeJsonForJob(PrintWriter wrt, RecentJobEvent jobEvent) throws IOException { - ExecutionGraph graph = jobmanager.getRecentExecutionGraph(jobEvent.getJobID()); - - //Serialize job to json - wrt.write("{"); - wrt.write("\"jobid\": \"" + jobEvent.getJobID() + "\","); - wrt.write("\"jobname\": \"" + jobEvent.getJobName()+"\","); - wrt.write("\"status\": \""+ jobEvent.getJobStatus() + "\","); - wrt.write("\"time\": " + jobEvent.getTimestamp()+","); - - // Serialize ManagementGraph to json - wrt.write("\"groupvertices\": ["); - boolean first = true; - - for (ExecutionJobVertex groupVertex : graph.getVerticesTopologically()) { - //Write seperator between json objects - if(first) { - first = false; - } else { - wrt.write(","); } - - wrt.write(JsonFactory.toJson(groupVertex)); - } - wrt.write("]"); - wrt.write("}"); +// ExecutionGraph graph = jobmanager.getRecentExecutionGraph(jobEvent.getJobID()); +// +// //Serialize job to json +// wrt.write("{"); +// wrt.write("\"jobid\": \"" + jobEvent.getJobID() + "\","); +// wrt.write("\"jobname\": \"" + jobEvent.getJobName()+"\","); +// wrt.write("\"status\": \""+ jobEvent.getJobStatus() + "\","); +// wrt.write("\"time\": " + jobEvent.getTimestamp()+","); +// +// // Serialize ManagementGraph to json +// wrt.write("\"groupvertices\": ["); +// boolean first = true; +// +// for (ExecutionJobVertex groupVertex : graph.getVerticesTopologically()) { +// //Write seperator between json objects +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// wrt.write(JsonFactory.toJson(groupVertex)); +// } +// wrt.write("]"); +// wrt.write("}"); } @@ -229,9 +229,262 @@ public class JobmanagerInfoServlet extends HttpServlet { */ private void writeJsonForArchivedJob(PrintWriter wrt, RecentJobEvent jobEvent) { - try { +// try { +// +// wrt.write("["); +// +// ManagementGraph jobManagementGraph = jobmanager.getManagementGraph(jobEvent.getJobID()); +// +// //Serialize job to json +// wrt.write("{"); +// wrt.write("\"jobid\": \"" + jobEvent.getJobID() + "\","); +// wrt.write("\"jobname\": \"" + jobEvent.getJobName()+"\","); +// wrt.write("\"status\": \""+ jobEvent.getJobStatus() + "\","); +// wrt.write("\"SCHEDULED\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.SCHEDULED) + ","); +// wrt.write("\"RUNNING\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.RUNNING) + ","); +// wrt.write("\"FINISHED\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.FINISHED) + ","); +// wrt.write("\"FAILED\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.FAILED) + ","); +// wrt.write("\"CANCELED\": "+ jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.CANCELED) + ","); +// wrt.write("\"CREATED\": " + jobmanager.getArchive().getJobTime(jobEvent.getJobID(), JobStatus.CREATED)+","); +// +// if (jobEvent.getJobStatus() == JobStatus.FAILED) { +// ManagementGraphIterator managementGraphIterator = new ManagementGraphIterator(jobManagementGraph,true); +// wrt.write("\"failednodes\": ["); +// HashSet map = new HashSet(); +// boolean first = true; +// while (managementGraphIterator.hasNext()) { +// ManagementVertex managementVertex = managementGraphIterator.next(); +// String instanceName = managementVertex.getInstanceName(); +// if (managementVertex.getExecutionState() == ExecutionState.FAILED && !map.contains(instanceName)) { +// if (first) { +// first = false; +// } else { +// wrt.write(","); +// } +// wrt.write("{"); +// wrt.write("\"node\": \"" + instanceName + "\","); +// wrt.write("\"message\": \"" + StringUtils.escapeHtml(managementVertex.getOptMessage()) + "\""); +// wrt.write("}"); +// map.add(instanceName); +// } +// } +// wrt.write("],"); +// } +// +// // Serialize ManagementGraph to json +// wrt.write("\"groupvertices\": ["); +// boolean first = true; +// for(ManagementGroupVertex groupVertex : jobManagementGraph.getGroupVerticesInTopologicalOrder()) { +// //Write seperator between json objects +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// wrt.write(groupVertex.toJson()); +// +// } +// wrt.write("],"); +// +// // write accumulators +// Map accMap = AccumulatorHelper.toResultMap(jobmanager.getAccumulators(jobEvent.getJobID())); +// +// wrt.write("\n\"accumulators\": ["); +// int i = 0; +// for( Entry accumulator : accMap.entrySet()) { +// wrt.write("{ \"name\": \""+accumulator.getKey()+" (" + accumulator.getValue().getClass().getName()+")\"," +// + " \"value\": \""+accumulator.getValue().toString()+"\"}\n"); +// if(++i < accMap.size()) { +// wrt.write(","); +// } +// } +// wrt.write("],\n"); +// +// wrt.write("\"groupverticetimes\": {"); +// first = true; +// for(ManagementGroupVertex groupVertex : jobManagementGraph.getGroupVerticesInTopologicalOrder()) { +// +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// // Calculate start and end time for groupvertex +// long started = Long.MAX_VALUE; +// long ended = 0; +// +// // Take earliest running state and latest endstate of groupmembers +// for(int j = 0; j < groupVertex.getNumberOfGroupMembers(); j++) { +// ManagementVertex vertex = groupVertex.getGroupMember(j); +// +// long running = jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.RUNNING); +// if(running != 0 && running < started) { +// started = running; +// } +// +// long finished = jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.FINISHED); +// long canceled = jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.CANCELED); +// long failed = jobmanager.getArchive().getVertexTime(jobEvent.getJobID(), vertex.getID(), ExecutionState.FAILED); +// +// if(finished != 0 && finished > ended) { +// ended = finished; +// } +// +// if(canceled != 0 && canceled > ended) { +// ended = canceled; +// } +// +// if(failed != 0 && failed > ended) { +// ended = failed; +// } +// +// } +// +// wrt.write("\""+groupVertex.getID()+"\": {"); +// wrt.write("\"groupvertexid\": \"" + groupVertex.getID() + "\","); +// wrt.write("\"groupvertexname\": \"" + groupVertex + "\","); +// wrt.write("\"STARTED\": "+ started + ","); +// wrt.write("\"ENDED\": "+ ended); +// wrt.write("}"); +// +// } +// +// wrt.write("}"); +// +// wrt.write("}"); +// +// +// wrt.write("]"); +// +// } catch (EofException eof) { // Connection closed by client +// LOG.info("Info server for jobmanager: Connection closed by client, EofException"); +// } catch (IOException ioe) { // Connection closed by client +// LOG.info("Info server for jobmanager: Connection closed by client, IOException"); +// } - wrt.write("["); +// wrt.write("["); +// +// ExecutionGraph graph = jobmanager.getRecentExecutionGraph(jobEvent.getJobID()); +// +// //Serialize job to json +// wrt.write("{"); +// wrt.write("\"jobid\": \"" + jobEvent.getJobID() + "\","); +// wrt.write("\"jobname\": \"" + jobEvent.getJobName()+"\","); +// wrt.write("\"status\": \""+ jobEvent.getJobStatus() + "\","); +// wrt.write("\"SCHEDULED\": "+ graph.getStatusTimestamp(JobStatus.CREATED) + ","); +// wrt.write("\"RUNNING\": "+ graph.getStatusTimestamp(JobStatus.RUNNING) + ","); +// wrt.write("\"FINISHED\": "+ graph.getStatusTimestamp(JobStatus.FINISHED) + ","); +// wrt.write("\"FAILED\": "+ graph.getStatusTimestamp(JobStatus.FAILED) + ","); +// wrt.write("\"CANCELED\": "+ graph.getStatusTimestamp(JobStatus.CANCELED) + ","); +// +// if (jobEvent.getJobStatus() == JobStatus.FAILED) { +// wrt.write("\"failednodes\": ["); +// boolean first = true; +// for (ExecutionVertex2 vertex : graph.getAllExecutionVertices()) { +// if (vertex.getExecutionState() == ExecutionState2.FAILED) { +// if (first) { +// first = false; +// } else { +// wrt.write(","); +// } +// wrt.write("{"); +// wrt.write("\"node\": \"" + vertex.getAssignedResource().getInstance().getInstanceConnectionInfo().hostname() + "\","); +// wrt.write("\"message\": \"" + StringUtils.escapeHtml(ExceptionUtils.stringifyException(vertex.getFailureCause())) + "\""); +// wrt.write("}"); +// } +// } +// wrt.write("],"); +// } +// +// // Serialize ManagementGraph to json +// wrt.write("\"groupvertices\": ["); +// boolean first = true; +// for (ExecutionJobVertex groupVertex : graph.getVerticesTopologically()) { +// //Write seperator between json objects +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// wrt.write(JsonFactory.toJson(groupVertex)); +// +// } +// wrt.write("],"); +// +// // write accumulators +// Map accMap = AccumulatorHelper.toResultMap(jobmanager.getAccumulators(jobEvent.getJobID())); +// +// wrt.write("\n\"accumulators\": ["); +// int i = 0; +// for( Entry accumulator : accMap.entrySet()) { +// wrt.write("{ \"name\": \""+accumulator.getKey()+" (" + accumulator.getValue().getClass().getName()+")\"," +// + " \"value\": \""+accumulator.getValue().toString()+"\"}\n"); +// if(++i < accMap.size()) { +// wrt.write(","); +// } +// } +// wrt.write("],\n"); +// +// wrt.write("\"groupverticetimes\": {"); +// first = true; +// for (ExecutionJobVertex groupVertex : graph.getVerticesTopologically()) { +// +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// // Calculate start and end time for groupvertex +// long started = Long.MAX_VALUE; +// long ended = 0; +// +// // Take earliest running state and latest endstate of groupmembers +// for (ExecutionVertex2 vertex : groupVertex.getTaskVertices()) { +// +// long running = vertex.getStateTimestamp(ExecutionState2.RUNNING); +// if (running != 0 && running < started) { +// started = running; +// } +// +// long finished = vertex.getStateTimestamp(ExecutionState2.FINISHED); +// long canceled = vertex.getStateTimestamp(ExecutionState2.CANCELED); +// long failed = vertex.getStateTimestamp(ExecutionState2.FAILED); +// +// if(finished != 0 && finished > ended) { +// ended = finished; +// } +// +// if(canceled != 0 && canceled > ended) { +// ended = canceled; +// } +// +// if(failed != 0 && failed > ended) { +// ended = failed; +// } +// +// } +// +// wrt.write("\""+groupVertex.getJobVertexId()+"\": {"); +// wrt.write("\"groupvertexid\": \"" + groupVertex.getJobVertexId() + "\","); +// wrt.write("\"groupvertexname\": \"" + groupVertex + "\","); +// wrt.write("\"STARTED\": "+ started + ","); +// wrt.write("\"ENDED\": "+ ended); +// wrt.write("}"); +// +// } +// +// wrt.write("}"); +// +// wrt.write("}"); +// +// +// wrt.write("]"); +// +// } catch (EofException eof) { // Connection closed by client +// LOG.info("Info server for jobmanager: Connection closed by client, EofException"); +// } catch (IOException ioe) { // Connection closed by client +// LOG.info("Info server for jobmanager: Connection closed by client, IOException"); +// } ExecutionGraph graph = jobmanager.getRecentExecutionGraph(jobEvent.getJobID()); @@ -370,81 +623,133 @@ public class JobmanagerInfoServlet extends HttpServlet { */ private void writeJsonUpdatesForJob(PrintWriter wrt, JobID jobId) { - try { - - List events = jobmanager.getEvents(jobId); - - //Serialize job to json - wrt.write("{"); - wrt.write("\"jobid\": \"" + jobId + "\","); - wrt.write("\"timestamp\": \"" + System.currentTimeMillis() + "\","); - wrt.write("\"recentjobs\": ["); - - boolean first = true; - for(RecentJobEvent rje: jobmanager.getRecentJobs()) { - if(first) { - first = false; - } else { - wrt.write(","); } - - wrt.write("\""+rje.getJobID().toString()+"\""); - } - - wrt.write("],"); - - wrt.write("\"vertexevents\": ["); - - first = true; - for (AbstractEvent event: events) { - - if (event instanceof ExecutionStateChangeEvent) { - - if(first) { - first = false; - } else { - wrt.write(","); } - - ExecutionStateChangeEvent vertexevent = (ExecutionStateChangeEvent) event; - wrt.write("{"); - wrt.write("\"vertexid\": \"" + vertexevent.getExecutionAttemptID() + "\","); - wrt.write("\"newstate\": \"" + vertexevent.getNewExecutionState() + "\","); - wrt.write("\"timestamp\": \"" + vertexevent.getTimestamp() + "\""); - wrt.write("}"); - } - } - - wrt.write("],"); - - wrt.write("\"jobevents\": ["); - - first = true; - for(AbstractEvent event: events) { - - if( event instanceof JobEvent) { - - if(first) { - first = false; - } else { - wrt.write(","); } - - JobEvent jobevent = (JobEvent) event; - wrt.write("{"); - wrt.write("\"newstate\": \"" + jobevent.getCurrentJobStatus() + "\","); - wrt.write("\"timestamp\": \"" + jobevent.getTimestamp() + "\""); - wrt.write("}"); - } - } - - wrt.write("]"); - - wrt.write("}"); - +// try { +// +// List events = jobmanager.getEvents(jobId); +// +// //Serialize job to json +// wrt.write("{"); +// wrt.write("\"jobid\": \"" + jobId + "\","); +// wrt.write("\"timestamp\": \"" + System.currentTimeMillis() + "\","); +// wrt.write("\"recentjobs\": ["); +// +// boolean first = true; +// for(RecentJobEvent rje: jobmanager.getRecentJobs()) { +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// wrt.write("\""+rje.getJobID().toString()+"\""); +// } +// +// wrt.write("],"); +// +// wrt.write("\"vertexevents\": ["); +// +// first = true; +// for(AbstractEvent event: events) { +// +// if(event instanceof ExecutionStateChangeEvent) { +// +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// ExecutionStateChangeEvent vertexevent = (ExecutionStateChangeEvent) event; +// wrt.write("{"); +// wrt.write("\"vertexid\": \"" + vertexevent.getVertexID() + "\","); +// wrt.write("\"newstate\": \"" + vertexevent.getNewExecutionState() + "\","); +// wrt.write("\"timestamp\": \"" + vertexevent.getTimestamp() + "\""); +// wrt.write("}"); +// } +// } +// +// wrt.write("],"); +// +// wrt.write("\"jobevents\": ["); +// +// first = true; +// for(AbstractEvent event: events) { +// +// if( event instanceof JobEvent) { +// +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// JobEvent jobevent = (JobEvent) event; +// wrt.write("{"); +// wrt.write("\"newstate\": \"" + jobevent.getCurrentJobStatus() + "\","); +// wrt.write("\"timestamp\": \"" + jobevent.getTimestamp() + "\""); +// wrt.write("}"); +// } +// } +// +// wrt.write("]"); +// +// wrt.write("}"); +// +// +// } catch (EofException eof) { // Connection closed by client +// LOG.info("Info server for jobmanager: Connection closed by client, EofException"); +// } catch (IOException ioe) { // Connection closed by client +// LOG.info("Info server for jobmanager: Connection closed by client, IOException"); +// } - } catch (EofException eof) { // Connection closed by client - LOG.info("Info server for jobmanager: Connection closed by client, EofException"); - } catch (IOException ioe) { // Connection closed by client - LOG.info("Info server for jobmanager: Connection closed by client, IOException"); - } +// first = true; +// for (AbstractEvent event: events) { +// +// if (event instanceof ExecutionStateChangeEvent) { +// +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// ExecutionStateChangeEvent vertexevent = (ExecutionStateChangeEvent) event; +// wrt.write("{"); +// wrt.write("\"vertexid\": \"" + vertexevent.getExecutionAttemptID() + "\","); +// wrt.write("\"newstate\": \"" + vertexevent.getNewExecutionState() + "\","); +// wrt.write("\"timestamp\": \"" + vertexevent.getTimestamp() + "\""); +// wrt.write("}"); +// } +// } +// +// wrt.write("],"); +// +// wrt.write("\"jobevents\": ["); +// +// first = true; +// for(AbstractEvent event: events) { +// +// if( event instanceof JobEvent) { +// +// if(first) { +// first = false; +// } else { +// wrt.write(","); } +// +// JobEvent jobevent = (JobEvent) event; +// wrt.write("{"); +// wrt.write("\"newstate\": \"" + jobevent.getCurrentJobStatus() + "\","); +// wrt.write("\"timestamp\": \"" + jobevent.getTimestamp() + "\""); +// wrt.write("}"); +// } +// } +// +// wrt.write("]"); +// +// wrt.write("}"); +// +// +// } catch (EofException eof) { // Connection closed by client +// LOG.info("Info server for jobmanager: Connection closed by client, EofException"); +// } catch (IOException ioe) { // Connection closed by client +// LOG.info("Info server for jobmanager: Connection closed by client, IOException"); +// } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java index 1469a4c4b6d7671f75c9d1cf5d23fbe77fe230b2..f35002a544ec6a0ff5a8e1466081cf307ec9f3f7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java @@ -39,8 +39,8 @@ public class JsonFactory { json.append("\"vertexname\": \"" + StringUtils.escapeHtml(vertex.getSimpleName()) + "\","); json.append("\"vertexstatus\": \"" + vertex.getExecutionState() + "\","); - AllocatedSlot slot = vertex.getCurrentAssignedResource(); - String instanceName = slot == null ? "(null)" : slot.getInstance().getInstanceConnectionInfo().getFQDNHostname(); + AllocatedSlot slot = vertex.getAssignedSlot(); + String instanceName = slot == null ? "(null)" : slot.getInstance().getPath(); json.append("\"vertexinstancename\": \"" + instanceName + "\""); json.append("}"); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java index 42a90664f86a0d787758837e90c9b353a1ab24e2..41d889ebcdd924ac10a89d67e9a3132edff98b88 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java @@ -32,14 +32,19 @@ import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import akka.actor.ActorRef; +import akka.pattern.Patterns; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.jobmanager.JobManager; import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; +import scala.concurrent.Await; +import scala.concurrent.Future; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,10 +62,10 @@ public class SetupInfoServlet extends HttpServlet { private Configuration globalC; - private JobManager2 jobmanager; + private ActorRef jobmanager; - public SetupInfoServlet(JobManager jm) { + public SetupInfoServlet(ActorRef jm) { globalC = GlobalConfiguration.getConfiguration(); this.jobmanager = jm; } @@ -100,51 +105,53 @@ public class SetupInfoServlet extends HttpServlet { private void writeTaskmanagers(HttpServletResponse resp) throws IOException { - List instances = new ArrayList(jobmanager.getInstanceManager().getAllRegisteredInstances().values()); - - Collections.sort(instances, INSTANCE_SORTER); - - JSONObject obj = new JSONObject(); - JSONArray array = new JSONArray(); - for (Instance instance : instances) { - JSONObject objInner = new JSONObject(); - - long time = new Date().getTime() - instance.getLastHeartBeat(); - - try { - objInner.put("inetAdress", instance.getInstanceConnectionInfo().getInetAdress()); - objInner.put("ipcPort", instance.getInstanceConnectionInfo().ipcPort()); - objInner.put("dataPort", instance.getInstanceConnectionInfo().dataPort()); - objInner.put("timeSinceLastHeartbeat", time / 1000); - objInner.put("slotsNumber", instance.getTotalNumberOfSlots()); - objInner.put("freeSlots", instance.getNumberOfAvailableSlots()); - objInner.put("cpuCores", instance.getResources().getNumberOfCPUCores()); - objInner.put("physicalMemory", instance.getResources().getSizeOfPhysicalMemory() >>> 20); - objInner.put("freeMemory", instance.getResources().getSizeOfJvmHeap() >>> 20); - objInner.put("managedMemory", instance.getResources().getSizeOfManagedMemory() >>> 20); - array.put(objInner); - } - catch (JSONException e) { - LOG.warn("Json object creation failed", e); - } - - } - try { - obj.put("taskmanagers", array); - } catch (JSONException e) { - LOG.warn("Json object creation failed", e); - } - - PrintWriter w = resp.getWriter(); - w.write(obj.toString()); +// List instances = new ArrayList(jobmanager.getInstanceManager().getAllRegisteredInstances().values()); +// +// Collections.sort(instances, INSTANCE_SORTER); +// +// JSONObject obj = new JSONObject(); +// JSONArray array = new JSONArray(); +// for (Instance instance : instances) { +// JSONObject objInner = new JSONObject(); +// +// long time = new Date().getTime() - instance.getLastHeartBeat(); +// +// try { +// objInner.put("inetAdress", instance.getInstanceConnectionInfo().getInetAdress()); +// objInner.put("ipcPort", instance.getInstanceConnectionInfo().ipcPort()); +// objInner.put("dataPort", instance.getInstanceConnectionInfo().dataPort()); +// objInner.put("timeSinceLastHeartbeat", time / 1000); +// objInner.put("slotsNumber", instance.getTotalNumberOfSlots()); +// objInner.put("freeSlots", instance.getNumberOfAvailableSlots()); +// objInner.put("cpuCores", instance.getResources().getNumberOfCPUCores()); +// objInner.put("physicalMemory", instance.getResources().getSizeOfPhysicalMemory() >>> 20); +// objInner.put("freeMemory", instance.getResources().getSizeOfJvmHeap() >>> 20); +// objInner.put("managedMemory", instance.getResources().getSizeOfManagedMemory() >>> 20); +// array.put(objInner); +// } +// catch (JSONException e) { +// LOG.warn("Json object creation failed", e); +// } +// obj.put("taskmanagers", array); +// +// PrintWriter w = resp.getWriter(); +// w.write(obj.toString()); +// }catch (JSONException e) { +// LOG.warn("Aggregated JSON object creation failed", e); +// }catch(IOException ioe){ +// throw ioe; +// }catch(Exception e){ +// throw new RuntimeException("Caught exception while requesting instances from jobmanager.", e); +// } + } // -------------------------------------------------------------------------------------------- - private static final Comparator INSTANCE_SORTER = new Comparator() { - @Override - public int compare(Instance o1, Instance o2) { - return o1.getInstanceConnectionInfo().compareTo(o2.getInstanceConnectionInfo()); - } - }; +// private static final Comparator INSTANCE_SORTER = new Comparator() { +// @Override +// public int compare(Instance o1, Instance o2) { +// return o1.getInstanceConnectionInfo().compareTo(o2.getInstanceConnectionInfo()); +// } +// }; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java index 59c8ac42039ef5dbbe23c3b841ef15b67e732119..44396449b552aea6559f4a6f812019e3e0c0f036 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/WebInfoServer.java @@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.runtime.jobmanager.JobManager2; +import org.apache.flink.runtime.jobmanager.JobManager; import org.eclipse.jetty.http.security.Constraint; import org.eclipse.jetty.security.ConstraintMapping; import org.eclipse.jetty.security.ConstraintSecurityHandler; @@ -73,7 +73,7 @@ public class WebInfoServer { * @throws IOException * Thrown, if the server setup failed for an I/O related reason. */ - public WebInfoServer(Configuration nepheleConfig, int port, JobManager2 jobmanager) throws IOException { + public WebInfoServer(Configuration nepheleConfig, int port, ActorRef jobmanager) throws IOException { this.port = port; // if no explicit configuration is given, use the global configuration diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java index 1d7af6588a94533ca575ef9215060a4831bb6518..f08ce0b535b04386706389017051e3d014cd87c5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/net/NetUtils.java @@ -28,8 +28,10 @@ package org.apache.flink.runtime.net; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.net.Inet4Address; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.NetworkInterface; import java.net.Socket; import java.net.SocketAddress; import java.net.URI; @@ -38,6 +40,7 @@ import java.net.UnknownHostException; import java.nio.channels.SocketChannel; import java.util.ArrayList; import java.util.Collection; +import java.util.Enumeration; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -328,4 +331,128 @@ public class NetUtils { } return hostNames; } + + /** + * The states of address detection mechanism. + * There is only a state transition if the current state failed to determine the address. + */ + private enum AddressDetectionState { + ADDRESS(50), //detect own IP based on the JobManagers IP address. Look for common prefix + FAST_CONNECT(50), //try to connect to the JobManager on all Interfaces and all their addresses. + //this state uses a low timeout (say 50 ms) for fast detection. + SLOW_CONNECT(1000), //same as FAST_CONNECT, but with a timeout of 1000 ms (1s). + HEURISTIC(0); + + + private int timeout; + AddressDetectionState(int timeout) { + this.timeout = timeout; + } + public int getTimeout() { + return timeout; + } + } + + /** + * Find out the TaskManager's own IP address. + */ + public static InetAddress resolveAddress(InetSocketAddress jobManagerAddress) throws IOException { + AddressDetectionState strategy = jobManagerAddress != null ? AddressDetectionState.ADDRESS: AddressDetectionState.HEURISTIC; + + while (true) { + Enumeration e = NetworkInterface.getNetworkInterfaces(); + while (e.hasMoreElements()) { + NetworkInterface n = e.nextElement(); + Enumeration ee = n.getInetAddresses(); + while (ee.hasMoreElements()) { + InetAddress i = ee.nextElement(); + switch (strategy) { + case ADDRESS: + if (hasCommonPrefix(jobManagerAddress.getAddress().getAddress(), i.getAddress())) { + if (tryToConnect(i, jobManagerAddress, strategy.getTimeout())) { + LOG.info("Determined " + i + " as the TaskTracker's own IP address"); + return i; + } + } + break; + case FAST_CONNECT: + case SLOW_CONNECT: + boolean correct = tryToConnect(i, jobManagerAddress, strategy.getTimeout()); + if (correct) { + LOG.info("Determined " + i + " as the TaskTracker's own IP address"); + return i; + } + break; + case HEURISTIC: + if(!i.isLinkLocalAddress() && !i.isLoopbackAddress() && i instanceof Inet4Address){ + LOG.warn("Hostname " + InetAddress.getLocalHost().getHostName() + " resolves to " + + "loopback address. Using instead " + i.getHostAddress() + " on network " + + "interface " + n.getName() + "."); + return i; + } + break; + default: + throw new RuntimeException("Unkown address detection strategy: " + strategy); + } + } + } + // state control + switch (strategy) { + case ADDRESS: + strategy = AddressDetectionState.FAST_CONNECT; + break; + case FAST_CONNECT: + strategy = AddressDetectionState.SLOW_CONNECT; + break; + case SLOW_CONNECT: + if(!InetAddress.getLocalHost().isLoopbackAddress()){ + return InetAddress.getLocalHost(); + }else { + strategy = AddressDetectionState.HEURISTIC; + break; + } + case HEURISTIC: + throw new RuntimeException("The TaskManager is unable to connect to the JobManager (Address: '"+jobManagerAddress+"')."); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Defaulting to detection strategy " + strategy); + } + } + } + + /** + * Checks if two addresses have a common prefix (first 2 bytes). + * Example: 192.168.???.??? + * Works also with ipv6, but accepts probably too many addresses + */ + private static boolean hasCommonPrefix(byte[] address, byte[] address2) { + return address[0] == address2[0] && address[1] == address2[1]; + } + + public static boolean tryToConnect(InetAddress fromAddress, SocketAddress toSocket, int timeout) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Trying to connect to JobManager (" + toSocket + ") from local address " + fromAddress + + " with timeout " + timeout); + } + boolean connectable = true; + Socket socket = null; + try { + socket = new Socket(); + SocketAddress bindP = new InetSocketAddress(fromAddress, 0); // 0 = let the OS choose the port on this + // machine + socket.bind(bindP); + socket.connect(toSocket, timeout); + } catch (Exception ex) { + LOG.info("Failed to connect to JobManager from address '" + fromAddress + "': " + ex.getMessage()); + if (LOG.isDebugEnabled()) { + LOG.debug("Failed with exception", ex); + } + connectable = false; + } finally { + if (socket != null) { + socket.close(); + } + } + return connectable; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java index e27562f46ed37527b2392bb5b0835522f326e16f..fbe91eda92236fd76f5ce3deb5d79ead52ba656c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/plugable/DeserializationDelegate.java @@ -37,6 +37,7 @@ public class DeserializationDelegate implements IOReadableWritable { public DeserializationDelegate(TypeSerializer serializer) { this.serializer = serializer; } + public void setInstance(T instance) { this.instance = instance; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/InstanceProfiler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/InstanceProfiler.java index 5433f75ed861cef4a40f6eaee3b2192566df0200..8eeda726db963a5aa2d317f35c1004fe2fc6ca81 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/InstanceProfiler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/InstanceProfiler.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.regex.Matcher; import java.util.regex.Pattern; +import akka.actor.ActorRef; import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.profiling.ProfilingException; import org.apache.flink.runtime.profiling.impl.types.InternalInstanceProfilingData; @@ -50,7 +51,7 @@ public class InstanceProfiler { private static final int PERCENT = 100; - private final InstanceConnectionInfo instanceConnectionInfo; + private final String instancePath; private long lastTimestamp = 0; @@ -76,10 +77,10 @@ public class InstanceProfiler { private long firstTimestamp; - public InstanceProfiler(InstanceConnectionInfo instanceConnectionInfo) + public InstanceProfiler(String instancePath) throws ProfilingException { - this.instanceConnectionInfo = instanceConnectionInfo; + this.instancePath = instancePath; this.firstTimestamp = System.currentTimeMillis(); // Initialize counters by calling generateProfilingData once and ignore the return value generateProfilingData(this.firstTimestamp); @@ -90,7 +91,7 @@ public class InstanceProfiler { final long profilingInterval = timestamp - lastTimestamp; final InternalInstanceProfilingData profilingData = new InternalInstanceProfilingData( - this.instanceConnectionInfo, (int) profilingInterval); + this.instancePath, (int) profilingInterval); updateCPUUtilization(profilingData); updateMemoryUtilization(profilingData); @@ -332,7 +333,7 @@ public class InstanceProfiler { final long profilingInterval = System.currentTimeMillis() - this.firstTimestamp; final InternalInstanceProfilingData profilingData = new InternalInstanceProfilingData( - this.instanceConnectionInfo, (int) profilingInterval); + this.instancePath, (int) profilingInterval); updateCPUUtilization(profilingData); updateMemoryUtilization(profilingData); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobManagerProfilerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobManagerProfilerImpl.java index 23b0ab9b1bf10096fa97d549cd57a41a3e7289af..1093526a31ecb6bcd5b93e1ed6c932041bb90e55 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobManagerProfilerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobManagerProfilerImpl.java @@ -162,8 +162,8 @@ public class JobManagerProfilerImpl implements JobManagerProfiler, ProfilerImplP profilingData.getSoftIrqCPU(), profilingData.getTotalMemory(), profilingData.getFreeMemory(), profilingData.getBufferedMemory(), profilingData.getCachedMemory(), profilingData .getCachedSwapMemory(), profilingData.getReceivedBytes(), profilingData.getTransmittedBytes(), - jobID, timestamp, timestamp - jobProfilingData.getProfilingStart(), profilingData - .getInstanceConnectionInfo().toString()); + jobID, timestamp, timestamp - jobProfilingData.getProfilingStart(), + profilingData.getInstancePath()); synchronized (this.registeredListeners) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java index 241d6cdd77eba4b4e57abad28edde4bef58c2b20..6eabe406ff5346756da967a56152e198e43aa2d0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java @@ -28,7 +28,6 @@ import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.instance.AllocatedSlot; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.profiling.impl.types.InternalInstanceProfilingData; import org.apache.flink.runtime.profiling.types.InstanceSummaryProfilingEvent; @@ -39,7 +38,8 @@ public class JobProfilingData { private final long profilingStart; - private final Map collectedInstanceProfilingData = new HashMap(); + private final Map collectedInstanceProfilingData = new + HashMap(); public JobProfilingData(ExecutionGraph executionGraph) { @@ -58,12 +58,12 @@ public class JobProfilingData { public boolean addIfInstanceIsAllocatedByJob(InternalInstanceProfilingData instanceProfilingData) { - for (ExecutionVertex executionVertex : this.executionGraph.getAllExecutionVertices()) { - AllocatedSlot slot = executionVertex.getCurrentAssignedResource(); - if (slot != null && slot.getInstance().getInstanceConnectionInfo().equals( - instanceProfilingData.getInstanceConnectionInfo())) + for (ExecutionVertex2 executionVertex : this.executionGraph.getAllExecutionVertices()) { + AllocatedSlot slot = executionVertex.getAssignedSlot(); + if (slot != null && slot.getInstance().getPath().equals( + instanceProfilingData.getInstancePath())) { - this.collectedInstanceProfilingData.put(instanceProfilingData.getInstanceConnectionInfo(), instanceProfilingData); + this.collectedInstanceProfilingData.put(instanceProfilingData.getInstancePath(), instanceProfilingData); return true; } } @@ -75,8 +75,8 @@ public class JobProfilingData { final Set tempSet = new HashSet(); - for (ExecutionVertex executionVertex : this.executionGraph.getAllExecutionVertices()) { - AllocatedSlot slot = executionVertex.getCurrentAssignedResource(); + for (ExecutionVertex2 executionVertex : this.executionGraph.getAllExecutionVertices()) { + AllocatedSlot slot = executionVertex.getAssignedSlot(); if (slot != null) { tempSet.add(slot.getInstance()); } @@ -95,7 +95,7 @@ public class JobProfilingData { final int numberOfInstances = this.collectedInstanceProfilingData.size(); - final Iterator instanceIterator = this.collectedInstanceProfilingData.keySet().iterator(); + final Iterator instanceIterator = this.collectedInstanceProfilingData.keySet().iterator(); long freeMemorySum = 0; long totalMemorySum = 0; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java index e1947923302223e7005d380dec7b853c12bab7ac..2f03f535f14d244827d9d272025cb3194a35e0da 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.profiling.impl.types.InternalInstanceProfilingDa import org.apache.flink.runtime.profiling.impl.types.ProfilingDataContainer; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.util.StringUtils; +import akka.actor.ActorRef; import java.io.IOException; import java.lang.management.ManagementFactory; @@ -66,7 +67,7 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro private final Map monitoredThreads = new HashMap(); - public TaskManagerProfilerImpl(InetAddress jobManagerAddress, InstanceConnectionInfo instanceConnectionInfo) + public TaskManagerProfilerImpl(InetAddress jobManagerAddress, String instancePath) throws ProfilingException { // Create RPC stub for communication with job manager's profiling component. @@ -90,7 +91,7 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro } // Create instance profiler - this.instanceProfiler = new InstanceProfiler(instanceConnectionInfo); + this.instanceProfiler = new InstanceProfiler(instancePath); // Set and trigger timer this.timerInterval = (long) (GlobalConfiguration.getInteger(ProfilingUtils.TASKMANAGER_REPORTINTERVAL_KEY, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInstanceProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInstanceProfilingData.java index f293fa91de248c707732a6dbc2d0f93d5cd92113..145d11c4f6c3b4dc5fb0b037b74d4c87b5c39b13 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInstanceProfilingData.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInstanceProfilingData.java @@ -22,11 +22,10 @@ import java.io.IOException; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; public class InternalInstanceProfilingData implements InternalProfilingData { - private InstanceConnectionInfo instanceConnectionInfo; + private String instancePath; private int profilingInterval; @@ -60,7 +59,7 @@ public class InternalInstanceProfilingData implements InternalProfilingData { this.freeMemory = -1; this.ioWaitCPU = -1; this.idleCPU = -1; - this.instanceConnectionInfo = new InstanceConnectionInfo(); + this.instancePath = ""; this.profilingInterval = -1; this.systemCPU = -1; this.totalMemory = -1; @@ -72,9 +71,9 @@ public class InternalInstanceProfilingData implements InternalProfilingData { this.transmittedBytes = -1; } - public InternalInstanceProfilingData(InstanceConnectionInfo instanceConnectionInfo, int profilingInterval) { + public InternalInstanceProfilingData(String instancePath, int profilingInterval) { - this.instanceConnectionInfo = instanceConnectionInfo; + this.instancePath = instancePath; this.profilingInterval = profilingInterval; this.freeMemory = -1; this.ioWaitCPU = -1; @@ -109,8 +108,8 @@ public class InternalInstanceProfilingData implements InternalProfilingData { return this.softIrqCPU; } - public InstanceConnectionInfo getInstanceConnectionInfo() { - return this.instanceConnectionInfo; + public String getInstancePath() { + return this.instancePath; } public int getProfilingInterval() { @@ -155,7 +154,7 @@ public class InternalInstanceProfilingData implements InternalProfilingData { this.freeMemory = in.readLong(); this.ioWaitCPU = in.readInt(); this.idleCPU = in.readInt(); - this.instanceConnectionInfo.read(in); + this.instancePath = in.readUTF(); this.profilingInterval = in.readInt(); this.systemCPU = in.readInt(); this.totalMemory = in.readLong(); @@ -176,7 +175,7 @@ public class InternalInstanceProfilingData implements InternalProfilingData { out.writeLong(this.freeMemory); out.writeInt(this.ioWaitCPU); out.writeInt(this.idleCPU); - this.instanceConnectionInfo.write(out); + out.writeUTF(instancePath); out.writeInt(this.profilingInterval); out.writeInt(this.systemCPU); out.writeLong(this.totalMemory); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 531c2e5d1f46b82d72d7be916f29f941ae631578..d9fb05601bd7f2d8f25d6f0697ed72859f37b0cb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.memorymanager.MemoryManager; +import org.apache.flink.runtime.messages.ExecutionGraphMessages; import org.apache.flink.runtime.profiling.TaskManagerProfiler; import org.apache.flink.util.ExceptionUtils; diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/ActorLogMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/ActorLogMessages.scala new file mode 100644 index 0000000000000000000000000000000000000000..9afc060aff4f9f692b8690283be8e0859d575690 --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/ActorLogMessages.scala @@ -0,0 +1,44 @@ +/* + * 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. + */ + +package org.apache.flink.runtime + +import _root_.akka.actor.Actor +import _root_.akka.event.LoggingAdapter + +trait ActorLogMessages { + self: Actor => + + override def receive: Receive = new Actor.Receive { + private val _receiveWithLogMessages = receiveWithLogMessages + + override def isDefinedAt(x: Any): Boolean = _receiveWithLogMessages.isDefinedAt(x) + + override def apply(x: Any):Unit = { + log.debug(s"Received message $x from ${self.sender}.") + val start = System.nanoTime() + _receiveWithLogMessages(x) + val duration = (System.nanoTime() - start) / 1000000 + log.debug(s"Handled message $x in $duration ms from ${self.sender}.") + } + } + + def receiveWithLogMessages: Receive + + protected def log: LoggingAdapter +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/serialization/IOReadableWritableSerializer.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/serialization/IOReadableWritableSerializer.scala new file mode 100644 index 0000000000000000000000000000000000000000..dff72bb9eda006150d28c5d0fcc5efb54da99b97 --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/serialization/IOReadableWritableSerializer.scala @@ -0,0 +1,61 @@ +/* + * 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. + */ + +package org.apache.flink.runtime.akka.serialization + +import akka.serialization.JSerializer +import org.apache.flink.core.io.IOReadableWritable +import org.apache.flink.runtime.io.network.serialization.{DataInputDeserializer, DataOutputSerializer} +import org.apache.flink.util.InstantiationUtil + +class IOReadableWritableSerializer extends JSerializer { + val INITIAL_BUFFER_SIZE = 8096 + + override protected def fromBinaryJava(bytes: Array[Byte], manifest: Class[_]): AnyRef = { + val in = new DataInputDeserializer(bytes, 0, bytes.length) + + val instance = InstantiationUtil.instantiate(manifest) + + if(!instance.isInstanceOf[IOReadableWritable]){ + throw new RuntimeException(s"Class $manifest is not of type IOReadableWritable.") + } + + val ioRW = instance.asInstanceOf[IOReadableWritable] + + ioRW.read(in) + + ioRW + } + + override def includeManifest: Boolean = true + + override def toBinary(o: AnyRef): Array[Byte] = { + if(!o.isInstanceOf[IOReadableWritable]){ + throw new RuntimeException("Object is not of type IOReadableWritable.") + } + + val ioRW = o.asInstanceOf[IOReadableWritable] + + val out = new DataOutputSerializer(INITIAL_BUFFER_SIZE) + ioRW.write(out) + + out.wrapAsByteBuffer().array() + } + + override def identifier: Int = 1337 +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/EventCollector.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/EventCollector.scala new file mode 100644 index 0000000000000000000000000000000000000000..8e97636a44502a373d47283fc5fd0de9ba0cf826 --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/EventCollector.scala @@ -0,0 +1,179 @@ +/* + * 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. + */ + +package org.apache.flink.runtime.jobmanager + + +import akka.actor.{Terminated, ActorRef, Actor, ActorLogging} +import org.apache.flink.runtime.ActorLogMessages +import org.apache.flink.runtime.event.job._ +import org.apache.flink.runtime.execution.ExecutionState +import org.apache.flink.runtime.executiongraph._ +import org.apache.flink.runtime.jobgraph.{JobVertexID, JobStatus, JobID} +import org.apache.flink.runtime.messages.ArchiveMessages.{ArchiveExecutionGraph, ArchiveJobEvent, ArchiveEvent} +import org.apache.flink.runtime.messages.EventCollectorMessages._ +import org.apache.flink.runtime.messages.ExecutionGraphMessages.{JobStatusChanged, ExecutionStateChanged} +import org.apache.hadoop.net.NetworkTopology +import scala.collection.convert.{WrapAsScala, DecorateAsJava} +import scala.concurrent.duration._ + +class EventCollector(val timerTaskInterval: Int) extends Actor with ActorLogMessages with ActorLogging with +DecorateAsJava with WrapAsScala { + import context.dispatcher + + val collectedEvents = collection.mutable.HashMap[JobID, List[AbstractEvent]]() + + val recentJobs = collection.mutable.HashMap[JobID, RecentJobEvent]() + + val recentExecutionGraphs = collection.mutable.HashMap[JobID, ExecutionGraph]() + + val archiveListeners = collection.mutable.HashSet[ActorRef]() + + val jobInformation = collection.mutable.HashMap[JobID, (String, Boolean, Long)]() + + override def preStart():Unit = { + startArchiveExpiredEvent() + } + + override def postStop(): Unit ={ + collectedEvents.clear() + recentJobs.clear() + recentExecutionGraphs.clear() + archiveListeners.clear() + jobInformation.clear() + } + + def startArchiveExpiredEvent():Unit = { + val schedulerDuration = FiniteDuration(2*timerTaskInterval, SECONDS) + context.system.scheduler.schedule(schedulerDuration, schedulerDuration, self, ArchiveExpiredEvents) + } + + override def receiveWithLogMessages: Receive = { + case ArchiveExpiredEvents => + val currentTime = System.currentTimeMillis() + + collectedEvents.retain{ + (jobID, events) => + val (outdatedElements, currentElements) = events.partition{ + event => event.getTimestamp + timerTaskInterval < currentTime + } + + outdatedElements foreach ( archiveEvent(jobID, _) ) + currentElements.nonEmpty + } + + recentJobs.retain{ + (jobID, recentJobEvent) => + import JobStatus._ + val status = recentJobEvent.getJobStatus + + // only remove jobs which have stopped running + if((status == FINISHED || status == CANCELED || status != FAILED) && + recentJobEvent.getTimestamp + timerTaskInterval < currentTime){ + archiveJobEvent(jobID, recentJobEvent) + archiveExecutionGraph(jobID, recentExecutionGraphs.remove(jobID).get) + jobInformation.remove(jobID) + false + }else{ + true + } + } + + case RequestJobEvents(jobID, includeManagementEvents) => + val events = collectedEvents.getOrElse(jobID, List()) + val filteredEvents = events filter { event => !event.isInstanceOf[ManagementEvent] || includeManagementEvents} + + sender() ! filteredEvents.asJava + + case RequestRecentJobs => + sender() ! recentJobs.values.asJavaCollection + + case RegisterJob(executionGraph, profilingAvailable, submissionTimestamp) => + val jobID = executionGraph.getJobID + + executionGraph.registerExecutionListener(self) + executionGraph.registerJobStatusListener(self) + jobInformation += jobID -> (executionGraph.getJobName, profilingAvailable, submissionTimestamp) + + case ExecutionStateChanged(jobID, vertexID, subtask, executionID, newExecutionState, optionalMessage) => + val timestamp = System.currentTimeMillis() + + recentExecutionGraphs.get(jobID) match { + case Some(graph) => + val vertex = graph.getJobVertex(vertexID) + val taskName = if(vertex != null) vertex.getJobVertex.getName else "(null)" + val totalNumberOfSubtasks = if(vertex != null) vertex.getParallelism else -1 + + + val vertexEvent = new VertexEvent(timestamp, vertexID, taskName, totalNumberOfSubtasks, subtask, executionID, + newExecutionState, optionalMessage) + + val events = collectedEvents.getOrElse(jobID, List()) + val executionStateChangeEvent = new ExecutionStateChangeEvent(timestamp, vertexID, subtask, + executionID, newExecutionState) + + collectedEvents += jobID -> (executionStateChangeEvent :: vertexEvent :: events) + case None => + log.warning(s"Could not find execution graph with jobID ${jobID}.") + } + + case JobStatusChanged(executionGraph, newJobStatus, optionalMessage) => + val jobID = executionGraph.getJobID() + + if(newJobStatus == JobStatus.RUNNING){ + this.recentExecutionGraphs += jobID -> executionGraph + } + + val currentTime = System.currentTimeMillis() + val (jobName, isProfilingEnabled, submissionTimestamp) = jobInformation(jobID) + recentJobs.put(jobID, new RecentJobEvent(jobID, jobName, newJobStatus, isProfilingEnabled, submissionTimestamp, + currentTime)) + + val events = collectedEvents.getOrElse(jobID, List()) + collectedEvents += jobID -> ((new JobEvent(currentTime, newJobStatus, optionalMessage))::events) + + case ProcessProfilingEvent(profilingEvent) => + val events = collectedEvents.getOrElse(profilingEvent.getJobID, List()) + collectedEvents += profilingEvent.getJobID -> (profilingEvent::events) + + case RegisterArchiveListener(actorListener) => + context.watch(actorListener) + archiveListeners += actorListener + + case Terminated(terminatedListener) => + archiveListeners -= terminatedListener + } + + private def archiveEvent(jobID: JobID, event: AbstractEvent): Unit = { + for(listener <- archiveListeners){ + listener ! ArchiveEvent(jobID, event) + } + } + + private def archiveJobEvent(jobID: JobID, event: RecentJobEvent): Unit = { + for(listener <- archiveListeners){ + listener ! ArchiveJobEvent(jobID, event) + } + } + + private def archiveExecutionGraph(jobID: JobID, graph: ExecutionGraph): Unit = { + for(listener <- archiveListeners){ + listener ! ArchiveExecutionGraph(jobID, graph) + } + } +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManagerCLIConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManagerCLIConfiguration.scala new file mode 100644 index 0000000000000000000000000000000000000000..d588f95f137d9e3dbfcd915ca3770056c699b8c7 --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManagerCLIConfiguration.scala @@ -0,0 +1,22 @@ +/* + * 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. + */ + +package org.apache.flink.runtime.jobmanager + +case class JobManagerCLIConfiguration(configDir: String = null) { +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala new file mode 100644 index 0000000000000000000000000000000000000000..f10613521f17534a5c3353e03228af0f7fab9541 --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/MemoryArchivist.scala @@ -0,0 +1,95 @@ +/* + * 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. + */ + +package org.apache.flink.runtime.jobmanager + +import akka.actor.{ActorLogging, Actor} +import org.apache.flink.runtime.ActorLogMessages +import org.apache.flink.runtime.event.job.{RecentJobEvent, AbstractEvent} +import org.apache.flink.runtime.executiongraph.ExecutionGraph +import org.apache.flink.runtime.jobgraph.JobID +import org.apache.flink.runtime.messages.ArchiveMessages._ + +import scala.collection.convert.DecorateAsJava +import scala.collection.mutable.ListBuffer + +class MemoryArchivist(private val max_entries: Int) extends Actor with ActorLogMessages with ActorLogging with +DecorateAsJava { + /** + * The map which stores all collected events until they are either + * fetched by the client or discarded. + */ + private val collectedEvents = collection.mutable.HashMap[JobID, ListBuffer[AbstractEvent]]() + + /** + * Map of recently started jobs with the time stamp of the last received job event. + */ + private val oldJobs = collection.mutable.HashMap[JobID, RecentJobEvent]() + + /** + * Map of execution graphs belonging to recently started jobs with the time stamp of the last received job event. + */ + private val graphs = collection.mutable.HashMap[JobID, ExecutionGraph]() + + + private val lru = collection.mutable.Queue[JobID]() + + override def receiveWithLogMessages: Receive = { + case ArchiveEvent(jobID, event) => + val list = collectedEvents.getOrElseUpdate(jobID, ListBuffer()) + list += event + cleanup(jobID) + + case ArchiveJobEvent(jobID, event) => + oldJobs.update(jobID, event) + cleanup(jobID) + + case ArchiveExecutionGraph(jobID, graph) => + graphs.update(jobID, graph) + cleanup(jobID) + + case GetJobs => + oldJobs.values.toSeq.asJava + + case GetJob(jobID) => + sender() ! oldJobs.get(jobID) + + case GetEvents(jobID) => + sender() ! collectedEvents.get(jobID) + + case GetExecutionGraph(jobID) => + sender() ! (graphs.get(jobID) match{ + case Some(graph) => graph + case None => akka.actor.Status.Failure(new IllegalArgumentException(s"Could not find execution graph for job " + + s"id $jobID.")) + }) + } + + def cleanup(jobID: JobID): Unit = { + if(!lru.contains(jobID)){ + lru.enqueue(jobID) + } + + while(lru.size > max_entries){ + val removedJobID = lru.dequeue() + collectedEvents.remove(removedJobID) + oldJobs.remove(removedJobID) + graphs.remove(removedJobID) + } + } +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala new file mode 100644 index 0000000000000000000000000000000000000000..c9d08b677db8d3f9d09fba7ff0d87c2d27a4cb76 --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ArchiveMessages.scala @@ -0,0 +1,33 @@ +/* + * 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. + */ + +package org.apache.flink.runtime.messages + +import org.apache.flink.runtime.event.job.{RecentJobEvent, AbstractEvent} +import org.apache.flink.runtime.executiongraph.ExecutionGraph +import org.apache.flink.runtime.jobgraph.JobID + +object ArchiveMessages { + case class ArchiveEvent(jobID: JobID, event: AbstractEvent) + case class ArchiveJobEvent(jobID: JobID, event: RecentJobEvent) + case class ArchiveExecutionGraph(jobID: JobID, graph: ExecutionGraph) + case class GetJob(jobID: JobID) + case class GetExecutionGraph(jobID: JobID) + case class GetEvents(jobID: JobID) + case object GetJobs +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/EventCollectorMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/EventCollectorMessages.scala new file mode 100644 index 0000000000000000000000000000000000000000..5989d1fe674e3d44b47c837cd06998b1e8d33aeb --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/EventCollectorMessages.scala @@ -0,0 +1,34 @@ +/* + * 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. + */ + +package org.apache.flink.runtime.messages + +import akka.actor.ActorRef +import org.apache.flink.runtime.executiongraph.ExecutionGraph +import org.apache.flink.runtime.jobgraph.JobID +import org.apache.flink.runtime.profiling.types.ProfilingEvent + +object EventCollectorMessages { + case class ProcessProfilingEvent(profilingEvent: ProfilingEvent) + case class RegisterArchiveListener(listener: ActorRef) + case class RequestJobEvents(jobID: JobID, includeManagementEvents: Boolean) + case class RegisterJob(executionGraph: ExecutionGraph, profilingAvailable: Boolean, submissionTimestamp: Long) + + case object ArchiveExpiredEvents + case object RequestRecentJobs +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala new file mode 100644 index 0000000000000000000000000000000000000000..e16a70bea7775eca750cf65fc63cff51df4db90f --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/ExecutionGraphMessages.scala @@ -0,0 +1,30 @@ +/* + * 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. + */ + +package org.apache.flink.runtime.messages + +import org.apache.flink.runtime.execution.{ExecutionState2} +import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGraph} +import org.apache.flink.runtime.jobgraph.{JobStatus, JobVertexID, JobID} + +object ExecutionGraphMessages { + case class ExecutionStateChanged(jobID: JobID, vertexID: JobVertexID, subtask: Int, executionID: ExecutionAttemptID, + newExecutionState: ExecutionState2, optionalMessage: String) + + case class JobStatusChanged(executionGraph: ExecutionGraph, newJobStatus: JobStatus, optionalMessage: String) +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala index 0bb816af7498a41dd4b99e315edb15cfe2242d63..9228850c0dd9c47aaa347fff099aed6941159cac 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala @@ -18,9 +18,12 @@ package org.apache.flink.runtime.messages -import org.apache.flink.runtime.instance.HardwareDescription +import org.apache.flink.runtime.jobgraph.{JobID, JobGraph} -object RegistrationMessages { - case class RegisterTaskManager(hardwareDescription: HardwareDescription) - case object AcknowledgeRegistration +object JobManagerMessages { + case class SubmitJob(jobGraph: JobGraph) + case class CancelJob(jobID: JobID) + + case object RequestInstances + case object RequestNumberRegisteredTaskManager } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala new file mode 100644 index 0000000000000000000000000000000000000000..b4f916ea60b57793691414114c2d6ec0160cc3a1 --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala @@ -0,0 +1,36 @@ +/* + * 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. + */ + +package org.apache.flink.runtime.messages + +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor +import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileRequest +import org.apache.flink.runtime.instance.InstanceID +import org.apache.flink.runtime.jobgraph.JobVertexID + +object TaskManagerMessages { + case class RequestLibraryCacheProfile(request: LibraryCacheProfileRequest) + case class CancelTask(jobVertexID: JobVertexID, subtaskIndex: Int) + case class SubmitTask(tasks: TaskDeploymentDescriptor) + + case class Heartbeat(instanceID: InstanceID) + + case object RegisterAtMaster + case object SendHeartbeat + case object AcknowledgeLibraryCacheUpdate +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerCLIConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerCLIConfiguration.scala new file mode 100644 index 0000000000000000000000000000000000000000..0e11a5fc279c0e5e91de5dd87b62669439f772af --- /dev/null +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManagerCLIConfiguration.scala @@ -0,0 +1,26 @@ +/* + * 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. + */ + +package org.apache.flink.runtime.taskmanager + +import org.apache.flink.runtime.instance.{InstanceID, HardwareDescription} + +object RegistrationMessages { + case class RegisterTaskManager(hardwareDescription: HardwareDescription, numberOfSlots: Int) + case class AcknowledgeRegistration(instanceID: InstanceID) +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairPairComparator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairPairComparator.java index 41f0bed6de5c4606d266537b6d6b2511159ebebc..fc2d7e0db9b9d0dcae2e13cbf49032b79629d0a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairPairComparator.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairPairComparator.java @@ -17,18 +17,122 @@ */ +import java.net.InetSocketAddress + import akka.actor._ -import org.apache.flink.configuration.Configuration +import akka.pattern.ask +import org.apache.flink.configuration.{GlobalConfiguration, ConfigConstants, Configuration} +import org.apache.flink.runtime.ActorLogMessages import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.messages.RegistrationMessages.{AcknowledgeRegistration, RegisteredTaskManager} +import org.apache.flink.runtime.execution.ExecutionState2 +import org.apache.flink.runtime.execution.librarycache.LibraryCacheUpdate +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID +import org.apache.flink.runtime.instance.{HardwareDescription, InstanceID} +import org.apache.flink.runtime.jobgraph.JobID +import org.apache.flink.runtime.jobmanager.JobManager +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager +import org.apache.flink.runtime.messages.RegistrationMessages.{RegisterTaskManager, AcknowledgeRegistration} +import org.apache.flink.runtime.messages.TaskManagerMessages.{AcknowledgeLibraryCacheUpdate, SendHeartbeat, Heartbeat, RegisterAtMaster} +import org.apache.flink.runtime.net.NetUtils +import org.apache.flink.runtime.util.EnvironmentInformation +import org.slf4j.LoggerFactory + +import scala.concurrent.duration._ +import scala.util.Failure + +class TaskManager(jobManagerURL: String, numberOfSlots: Int, memorySize: Long, + pageSize: Int) extends Actor with ActorLogMessages with +ActorLogging { + import context.dispatcher + import AkkaUtils.FUTURE_TIMEOUT + + val REGISTRATION_DELAY = 0 seconds + val REGISTRATION_INTERVAL = 10 seconds + val MAX_REGISTRATION_ATTEMPTS = 1 + val HEARTBEAT_INTERVAL = 200 millisecond + + + val memoryManager = new DefaultMemoryManager(memorySize, numberOfSlots, pageSize) + + val hardwareDescription = HardwareDescription.extractFromSystem(memoryManager.getMemorySize) + + var registrationScheduler: Option[Cancellable] = None + var registrationAttempts: Int = 0 + var registered: Boolean = false + var currentJobManager = ActorRef.noSender + var instanceID: InstanceID = null; + + override def preStart(): Unit = { + tryJobManagerRegistration() + } + + def tryJobManagerRegistration(): Unit = { + registrationAttempts = 0 + import context.dispatcher + registrationScheduler = Some(context.system.scheduler.schedule(REGISTRATION_DELAY, REGISTRATION_INTERVAL, + self, RegisterAtMaster)) + } + + + override def receiveWithLogMessages: Receive = { + case RegisterAtMaster => + registrationAttempts += 1 + + if(registered){ + registrationScheduler.foreach(_.cancel()) + } else if(registrationAttempts <= MAX_REGISTRATION_ATTEMPTS){ + val jobManagerURL = getJobManagerURL + + log.info(s"Try to register at master ${jobManagerURL}. ${registrationAttempts}. Attempt") + val jobManager = context.actorSelection(jobManagerURL) + + jobManager ! RegisterTaskManager(hardwareDescription, numberOfSlots) + }else{ + log.error("TaskManager could not register at JobManager."); + throw new RuntimeException("TaskManager could not register at JobManager"); + } + + case AcknowledgeRegistration(id) => + registered = true + currentJobManager = sender() + instanceID = id + val jobManagerAddress = currentJobManager.path.toString() + log.info(s"TaskManager successfully registered at JobManager $jobManagerAddress.") + context.system.scheduler.schedule(HEARTBEAT_INTERVAL, HEARTBEAT_INTERVAL, self, SendHeartbeat) + + case SendHeartbeat => + currentJobManager ! Heartbeat(instanceID) + + case x:LibraryCacheUpdate => + log.info(s"Registered library ${x.getLibraryFileName}.") + sender() ! AcknowledgeLibraryCacheUpdate + + } + + def notifyExecutionStateChange(jobID: JobID, executionID: ExecutionAttemptID, executionState : ExecutionState2, + message: String): Unit = { + val futureResponse = currentJobManager ? new TaskExecutionState(jobID, executionID, executionState, message) + + futureResponse.onComplete{ + x => + x match { + case Failure(ex) => + log.error(ex, "Error sending task state update to JobManager.") + case _ => + } + if(executionState == ExecutionState2.FINISHED || executionState == ExecutionState2.CANCELED || executionState == + ExecutionState2.FAILED){ + unregisterTask(executionID) + } + } + } -import org.apache.flink.api.common.typeutils.TypePairComparator; + def unregisterTask(executionID: ExecutionAttemptID): Unit = { - override def receive: Receive = { - case AcknowledgeRegistration => - println("Got registered at " + sender().toString()) - Thread.sleep(1000) - self ! PoisonPill + } + + private def getJobManagerURL: String = { + JobManager.getAkkaURL(jobManagerURL) } } @@ -37,14 +141,92 @@ public class IntPairPairComparator extends TypePairComparator private int key; - def startActorSystemAndActor(systemName: String, hostname: String, port: Int, actorName: String, - configuration: Configuration) = { - val actorSystem = AkkaUtils.createActorSystem(systemName, hostname, port, configuration) - startActor(actorSystem, actorName) + val LOG = LoggerFactory.getLogger(classOf[TaskManager]) + val FAILURE_RETURN_CODE = -1 + + def main(args: Array[String]): Unit = { + val (hostname, port, configuration) = initialize(args) + + val taskManagerSystem = startActorSystemAndActor(hostname, port, configuration) + taskManagerSystem.awaitTermination() + } + + private def initialize(args: Array[String]):(String, Int, Configuration) = { + val parser = new scopt.OptionParser[TaskManagerCLIConfiguration]("taskmanager"){ + head("flink task manager") + opt[String]("configDir") action { (x, c) => + c.copy(configDir = x) + } text("Specify configuration directory.") + opt[String]("tempDir") action { (x, c) => + c.copy(tmpDir = x) + } text("Specify temporary directory.") + } + + + parser.parse(args, TaskManagerCLIConfiguration()) map { + config => + GlobalConfiguration.loadConfiguration(config.configDir) + + val configuration = GlobalConfiguration.getConfiguration() + + if(config.tmpDir != null && GlobalConfiguration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, + null) == null){ + configuration.setString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, config.tmpDir) + } + + val jobManagerHostname = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null); + val jobManagerPort = configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, + ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT) + + val jobManagerAddress = new InetSocketAddress(jobManagerHostname, jobManagerPort); + + val port = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0) + val hostname = NetUtils.resolveAddress(jobManagerAddress).getHostName; + + (hostname, port, configuration) + } getOrElse { + LOG.error("CLI parsing failed. Usage: " + parser.usage) + sys.exit(FAILURE_RETURN_CODE) + } + } + + def startActorSystemAndActor(hostname: String, port: Int, configuration: Configuration) = { + val actorSystem = AkkaUtils.createActorSystem(hostname, port, configuration) + startActor(actorSystem, configuration) actorSystem } - def startActor(actorSystem: ActorSystem, actorName: String): ActorRef = { - actorSystem.actorOf(Props(classOf[TaskManager]), actorName); + def startActor(actorSystem: ActorSystem, configuration: Configuration): ActorRef = { + val jobManagerAddress = configuration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null); + val jobManagerRPCPort = configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, + ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT); + + if(jobManagerAddress == null){ + throw new RuntimeException("JobManager address has not been specified in the configuration.") + } + + val jobManagerURL = jobManagerAddress + ":" + jobManagerRPCPort + val slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1) + + val numberOfSlots = if(slots > 0) slots else 1 + + val configuredMemory:Long = configuration.getInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1) + + val memorySize = if(configuredMemory > 0){ + configuredMemory << 20 + } else{ + val fraction = configuration.getFloat(ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY, + ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION) + (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag * fraction).toLong + } + + val pageSize = configuration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE) + + actorSystem.actorOf(Props(classOf[TaskManager], jobManagerURL, numberOfSlots, memorySize, pageSize), "taskmanager"); + } + + def getAkkaURL(address: String): String = { + s"akka.tcp://flink@${address}/user/taskmanager" } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/impl/InstanceProfilerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/impl/InstanceProfilerTest.java index 6b40481e8fe2a578ae55c7a23e2df64bfffe7d1c..52ca4081f0cbe365bb50f9e895ef83c493fdac0a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/impl/InstanceProfilerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/profiling/impl/InstanceProfilerTest.java @@ -32,9 +32,9 @@ import java.io.BufferedReader; import java.io.FileReader; import java.net.InetAddress; +import akka.actor.ActorRef; import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.profiling.ProfilingException; -import org.apache.flink.runtime.profiling.impl.InstanceProfiler; import org.apache.flink.runtime.profiling.impl.types.InternalInstanceProfilingData; import org.junit.Before; import org.junit.Test; @@ -127,7 +127,7 @@ public class InstanceProfilerTest { PowerMockito.mockStatic(System.class); when(System.currentTimeMillis()).thenReturn(0L); - this.out = new InstanceProfiler(this.infoMock); + this.out = new InstanceProfiler(ActorRef.noSender()); } @Test diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml index 5139fb59e84a936caa4a7b38ecaa3dee0bed4070..20a7ba3fda074b310793187954b694723b8e640c 100644 --- a/flink-scala/pom.xml +++ b/flink-scala/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 diff --git a/flink-test-utils/pom.xml b/flink-test-utils/pom.xml index b3cbae633209774281854b58fbab275735b29923..2dd45ad9c899aa161e1015abbfe7ae972f896264 100644 --- a/flink-test-utils/pom.xml +++ b/flink-test-utils/pom.xml @@ -17,7 +17,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index cb95c95d26815011c1ae57053038dfc8637f51f2..924258059666e6486cfbf184d430842d0d34d690 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -18,7 +18,7 @@ specific language governing permissions and limitations under the License. --> - 4.0.0 diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/BooleanValue.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/BooleanValue.java index 0629bc51fa81c37bc3b42a67c0cc057367c6b8df..9f1c18be0bface4343c3f5ca654029cf0ed02b86 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/BooleanValue.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/BooleanValue.java @@ -17,14 +17,21 @@ */ -import akka.actor.{ExtendedActorSystem, ActorSystem} +import akka.actor.ActorSystem +import akka.util.Timeout import com.typesafe.config.ConfigFactory import org.apache.flink.configuration.{ConfigConstants, Configuration} +import org.apache.flink.core.io.IOReadableWritable +import org.apache.flink.runtime.akka.serialization.IOReadableWritableSerializer +import scala.concurrent.duration._ object AkkaUtils { - def createActorSystem(name: String, host: String, port: Int, configuration: Configuration): ActorSystem = { + implicit val FUTURE_TIMEOUT: Timeout = 1 minute + implicit val AWAIT_DURATION: Duration = 1 minute + + def createActorSystem(host: String, port: Int, configuration: Configuration): ActorSystem = { val akkaConfig = ConfigFactory.parseString(AkkaUtils.getConfigString(host, port, configuration)) - val actorSystem = ActorSystem.create(name, akkaConfig) + val actorSystem = ActorSystem.create("flink", akkaConfig) actorSystem } @@ -52,6 +59,8 @@ object AkkaUtils { val logLifecycleEvents = if(lifecycleEvents) "on" else "off" + val ioRWSerializerClass = classOf[IOReadableWritableSerializer].getCanonicalName + val ioRWClass = classOf[IOReadableWritable].getCanonicalName s"""akka.daemonic = on |akka.loggers = ["akka.event.slf4j.Slf4jLogger"] @@ -77,6 +86,12 @@ object AkkaUtils { |akka.remote.log-remote-lifecycle-events = $logLifecycleEvents |akka.log-dead-letters = $logLifecycleEvents |akka.log-dead-letters-during-shutdown = $logLifecycleEvents + |akka.actor.serializers { + | IOReadableWritable = "$ioRWSerializerClass" + |} + |akka.actor.serialization-bindings { + | "$ioRWClass" = IOReadableWritable + |} """.stripMargin } } diff --git a/pom.xml b/pom.xml index 3a8272a4bc955e8eeaede678fdf9c1175f379c40..f877849c69ce8efc754dab5bb0d85b075d1f899d 100644 --- a/pom.xml +++ b/pom.xml @@ -18,7 +18,7 @@ specific language governing permissions and limitations under the License. --> + xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-taskManager"> org.apache @@ -60,11 +60,12 @@ under the License. flink-compiler flink-examples flink-clients - flink-tests - flink-test-utils - flink-addons - flink-quickstart - flink-dist + + + + + + testing