Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/292#discussion_r23175580 --- Diff: flink-yarn/src/main/java/org/apache/flink/yarn/FlinkYarnCluster.java --- @@ -0,0 +1,363 @@ +/* + * 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.yarn; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; + +import static akka.pattern.Patterns.ask; + +import akka.actor.Props; +import akka.util.Timeout; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.net.NetUtils; +import org.apache.flink.runtime.yarn.AbstractFlinkYarnCluster; +import org.apache.flink.runtime.yarn.FlinkYarnClusterStatus; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.service.Service; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.None$; +import scala.Some; +import scala.concurrent.Await; +import scala.concurrent.Awaitable; +import scala.concurrent.Future; +import scala.concurrent.duration.Duration; +import scala.concurrent.duration.FiniteDuration; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + + +public class FlinkYarnCluster extends AbstractFlinkYarnCluster { + private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnCluster.class); + + private static final int POLLING_THREAD_INTERVAL_MS = 1000; + + private YarnClient yarnClient; + private Thread actorRunner; + private Thread clientShutdownHook = new ClientShutdownHook(); + private PollingThread pollingRunner; + private Configuration hadoopConfig; + // (HDFS) location of the files required to run on YARN. Needed here to delete them on shutdown. + private Path sessionFilesDir; + private InetSocketAddress jobManagerAddress; + + //---------- Class internal fields ------------------- + + private ActorSystem actorSystem; + private ActorRef applicationClient; + private ApplicationReport intialAppReport; + private static FiniteDuration akkaDuration = Duration.apply(5, TimeUnit.SECONDS); + private static Timeout akkaTimeout = Timeout.durationToTimeout(akkaDuration); + + public FlinkYarnCluster(final YarnClient yarnClient, final ApplicationId appId, + Configuration hadoopConfig, Path sessionFilesDir) throws IOException, YarnException { + this.yarnClient = yarnClient; + this.hadoopConfig = hadoopConfig; + this.sessionFilesDir = sessionFilesDir; + + // get one application report manually + intialAppReport = yarnClient.getApplicationReport(appId); + String jobManagerHost = intialAppReport.getHost(); + int jobManagerPort = intialAppReport.getRpcPort(); + this.jobManagerAddress = new InetSocketAddress(jobManagerHost, jobManagerPort); + + // 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 = YarnUtils.createActorSystem(ownHostname.getCanonicalHostName(), 0, GlobalConfiguration.getConfiguration()); // set port automatically. + + // start application client + LOG.info("Start application client."); + + applicationClient = actorSystem.actorOf(Props.create(ApplicationClient.class)); + + // instruct ApplicationClient to start a periodical status polling + applicationClient.tell(new Messages.LocalRegisterClient(jobManagerHost + ":" + jobManagerPort), applicationClient); + + + // add hook to ensure proper shutdown + Runtime.getRuntime().addShutdownHook(clientShutdownHook); + + 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(); + } + + // -------------------------- Interaction with the cluster ------------------------ + + @Override + public InetSocketAddress getJobManagerAddress() { + return jobManagerAddress; + } + + @Override + public String getWebInterfaceURL() { + return this.intialAppReport.getTrackingUrl(); + } + + + @Override + public FlinkYarnClusterStatus getClusterStatus() { + if(hasBeenStopped()) { + throw new RuntimeException("The FlinkYarnCluster has alread been stopped"); + } + Future<Object> clusterStatusOption = ask(applicationClient, Messages.LocalGetYarnClusterStatus$.MODULE$, akkaTimeout); + Object clusterStatus = awaitUtil(clusterStatusOption, "Unable to get Cluster status from Application Client"); + if(clusterStatus instanceof None$) { + return null; + } else if(clusterStatus instanceof Some) { + return (FlinkYarnClusterStatus) (((Some) clusterStatus).get()); + } else { + throw new RuntimeException("Unexpected type: "+clusterStatus.getClass().getCanonicalName()); + } + } + + @Override + public boolean hasFailed() { + if(pollingRunner == null) { + LOG.warn("FlinkYarnCluster.hasFailed() has been called on an uninitialized cluster." + + "The system might be in an erroneous state"); + } + ApplicationReport lastReport = pollingRunner.getLastReport(); + if(lastReport == null) { + LOG.warn("FlinkYarnCluster.hasFailed() has been called on a cluster. that didn't receive a status so far." + + "The system might be in an erroneous state"); + return false; + } else { + return (lastReport.getYarnApplicationState() == YarnApplicationState.FAILED || + lastReport.getYarnApplicationState() == YarnApplicationState.KILLED); + } + } + + @Override + public String getDiagnostics() { + if (!hasFailed()) { + LOG.warn("getDiagnostics() called for cluster which is not in failed state"); + } + ApplicationReport lastReport = pollingRunner.getLastReport(); + if (lastReport == null) { + LOG.warn("Last report is null"); + return null; + } else { + return lastReport.getDiagnostics(); + } + } + + @Override + public List<String> getNewMessages() { + if(hasBeenStopped()) { + throw new RuntimeException("The FlinkYarnCluster has alread been stopped"); + } + List<String> ret = new ArrayList<String>(); + // get messages from ApplicationClient (locally) + while(true) { + Future<Object> messageOptionFuture = ask(applicationClient, Messages.LocalGetYarnMessage$.MODULE$, akkaTimeout); + Object messageOption = awaitUtil(messageOptionFuture, "Error getting new messages from Appliation Client"); + if(messageOption instanceof None$) { + break; + } else if(messageOption instanceof org.apache.flink.yarn.Messages.YarnMessage) { + Messages.YarnMessage msg = (Messages.YarnMessage) messageOption; + ret.add("["+msg.date()+"] "+msg.message()); + } else { + LOG.warn("LocalGetYarnMessage returned unexpected type: "+messageOption); + } + } + return ret; + } + + private static <T> T awaitUtil(Awaitable<T> awaitable, String message) { --- End diff -- That is exactly what AkkaUtils.ask does. We could reduce code redundancy here.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---