@BeforeClass public static void setupJobManager() { Configuration config = new Configuration(); int port = NetUtils.getAvailablePort(); config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost"); config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port); scala.Option<Tuple2<String, Object>> listeningAddress = scala.Option.apply(new Tuple2<String, Object>("localhost", port)); jobManagerSystem = AkkaUtils.createActorSystem(config, listeningAddress); ActorRef jobManagerActorRef = JobManager.startJobManagerActors( config, jobManagerSystem, StreamingMode.BATCH_ONLY, JobManager.class, MemoryArchivist.class) ._1(); try { LeaderRetrievalService lrs = LeaderRetrievalUtils.createLeaderRetrievalService(config); jmGateway = LeaderRetrievalUtils.retrieveLeaderGateway(lrs, jobManagerSystem, timeout); } catch (Exception e) { fail("Could not retrieve the JobManager gateway. " + e.getMessage()); } }
private ActorRef getJobManager() throws IOException { final Configuration configuration = GlobalConfiguration.getConfiguration(); ActorSystem actorSystem; try { final scala.Tuple2<String, Object> systemEndpoint = new scala.Tuple2<String, Object>("", 0); actorSystem = AkkaUtils.createActorSystem( configuration, new Some<scala.Tuple2<String, Object>>(systemEndpoint)); } catch (final Exception e) { throw new RuntimeException("Could not start actor system to communicate with JobManager", e); } return JobManager.getJobManagerActorRef( new InetSocketAddress(this.jobManagerHost, this.jobManagerPort), actorSystem, AkkaUtils.getLookupTimeout(configuration)); }
/** * Connect the FlinkYarnCluster to the ApplicationMaster. * * <p>Detached YARN sessions don't need to connect to the ApplicationMaster. Detached per job YARN * sessions need to connect until the required number of TaskManagers have been started. * * @throws IOException */ public void connectToCluster() throws IOException { if (isConnected) { throw new IllegalStateException("Can not connect to the cluster again"); } // start actor system LOG.info("Start actor system."); InetAddress ownHostname = NetUtils.resolveAddress( jobManagerAddress); // find name of own public interface, able to connect to the JM actorSystem = AkkaUtils.createActorSystem( flinkConfig, new Some(new Tuple2<String, Integer>(ownHostname.getCanonicalHostName(), 0))); // start application client LOG.info("Start application client."); applicationClient = actorSystem.actorOf( Props.create(ApplicationClient.class, flinkConfig), "applicationClient"); // instruct ApplicationClient to start a periodical status polling applicationClient.tell( new Messages.LocalRegisterClient(this.jobManagerAddress), applicationClient); actorRunner = new Thread( new Runnable() { @Override public void run() { // blocks until ApplicationMaster has been stopped actorSystem.awaitTermination(); // get final application report try { ApplicationReport appReport = yarnClient.getApplicationReport(appId); LOG.info( "Application " + appId + " finished with state " + appReport.getYarnApplicationState() + " and final state " + appReport.getFinalApplicationStatus() + " at " + appReport.getFinishTime()); if (appReport.getYarnApplicationState() == YarnApplicationState.FAILED || appReport.getYarnApplicationState() == YarnApplicationState.KILLED) { LOG.warn("Application failed. Diagnostics " + appReport.getDiagnostics()); LOG.warn( "If log aggregation is activated in the Hadoop cluster, we recommend to retrieve " + "the full application log using this command:\n" + "\tyarn logs -applicationId " + appReport.getApplicationId() + "\n" + "(It sometimes takes a few seconds until the logs are aggregated)"); } } catch (Exception e) { LOG.warn("Error while getting final application report", e); } } }); actorRunner.setDaemon(true); actorRunner.start(); pollingRunner = new PollingThread(yarnClient, appId); pollingRunner.setDaemon(true); pollingRunner.start(); Runtime.getRuntime().addShutdownHook(clientShutdownHook); isConnected = true; }
@Test public void testJobManagerProcessFailure() throws Exception { // Config final int numberOfJobManagers = 2; final int numberOfTaskManagers = 2; final int numberOfSlotsPerTaskManager = 2; assertEquals(PARALLELISM, numberOfTaskManagers * numberOfSlotsPerTaskManager); // Setup // Test actor system ActorSystem testActorSystem; // Job managers final JobManagerProcess[] jmProcess = new JobManagerProcess[numberOfJobManagers]; // Task managers final ActorSystem[] tmActorSystem = new ActorSystem[numberOfTaskManagers]; // Leader election service LeaderRetrievalService leaderRetrievalService = null; // Coordination between the processes goes through a directory File coordinateTempDir = null; try { final Deadline deadline = TestTimeOut.fromNow(); // Coordination directory coordinateTempDir = createTempDirectory(); // Job Managers Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig( ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath()); // Start first process jmProcess[0] = new JobManagerProcess(0, config); jmProcess[0].createAndStart(); // Task manager configuration config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4); config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); // Start the task manager process for (int i = 0; i < numberOfTaskManagers; i++) { tmActorSystem[i] = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig()); TaskManager.startTaskManagerComponentsAndActor( config, tmActorSystem[i], "localhost", Option.<String>empty(), Option.<LeaderRetrievalService>empty(), false, TaskManager.class); } // Test actor system testActorSystem = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig()); jmProcess[0].getActorRef(testActorSystem, deadline.timeLeft()); // Leader listener TestingListener leaderListener = new TestingListener(); leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config); leaderRetrievalService.start(leaderListener); // Initial submission leaderListener.waitForNewLeader(deadline.timeLeft().toMillis()); String leaderAddress = leaderListener.getAddress(); UUID leaderId = leaderListener.getLeaderSessionID(); // Get the leader ref ActorRef leaderRef = AkkaUtils.getActorRef(leaderAddress, testActorSystem, deadline.timeLeft()); ActorGateway leaderGateway = new AkkaActorGateway(leaderRef, leaderId); // Wait for all task managers to connect to the leading job manager JobManagerActorTestUtils.waitForTaskManagers( numberOfTaskManagers, leaderGateway, deadline.timeLeft()); final File coordinateDirClosure = coordinateTempDir; final Throwable[] errorRef = new Throwable[1]; // we trigger program execution in a separate thread Thread programTrigger = new Thread("Program Trigger") { @Override public void run() { try { testJobManagerFailure(ZooKeeper.getConnectString(), coordinateDirClosure); } catch (Throwable t) { t.printStackTrace(); errorRef[0] = t; } } }; // start the test program programTrigger.start(); // wait until all marker files are in place, indicating that all tasks have started AbstractTaskManagerProcessFailureRecoveryTest.waitForMarkerFiles( coordinateTempDir, READY_MARKER_FILE_PREFIX, PARALLELISM, deadline.timeLeft().toMillis()); // Kill one of the job managers and trigger recovery jmProcess[0].destroy(); jmProcess[1] = new JobManagerProcess(1, config); jmProcess[1].createAndStart(); jmProcess[1].getActorRef(testActorSystem, deadline.timeLeft()); // we create the marker file which signals the program functions tasks that they can complete AbstractTaskManagerProcessFailureRecoveryTest.touchFile( new File(coordinateTempDir, PROCEED_MARKER_FILE)); programTrigger.join(deadline.timeLeft().toMillis()); // We wait for the finish marker file. We don't wait for the program trigger, because // we submit in detached mode. AbstractTaskManagerProcessFailureRecoveryTest.waitForMarkerFiles( coordinateTempDir, FINISH_MARKER_FILE_PREFIX, 1, deadline.timeLeft().toMillis()); // check that the program really finished assertFalse("The program did not finish in time", programTrigger.isAlive()); // check whether the program encountered an error if (errorRef[0] != null) { Throwable error = errorRef[0]; error.printStackTrace(); fail( "The program encountered a " + error.getClass().getSimpleName() + " : " + error.getMessage()); } } catch (Throwable t) { // Print early (in some situations the process logs get too big // for Travis and the root problem is not shown) t.printStackTrace(); for (JobManagerProcess p : jmProcess) { if (p != null) { p.printProcessLog(); } } throw t; } finally { for (int i = 0; i < numberOfTaskManagers; i++) { if (tmActorSystem[i] != null) { tmActorSystem[i].shutdown(); } } if (leaderRetrievalService != null) { leaderRetrievalService.stop(); } for (JobManagerProcess jmProces : jmProcess) { if (jmProces != null) { jmProces.destroy(); } } // Delete coordination directory if (coordinateTempDir != null) { try { FileUtils.deleteDirectory(coordinateTempDir); } catch (Throwable ignored) { } } } }