diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml index 8423e70..0187aa9 100644 --- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml +++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml @@ -123,6 +123,13 @@ + hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/target + /share/hadoop/${hadoop.component}/sources + + *-sources.jar + + + hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/target /share/hadoop/${hadoop.component}/sources diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index a4446e5..450ea54 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -415,9 +415,6 @@ Release 2.4.0 - UNRELEASED HADOOP-10208. Remove duplicate initialization in StringUtils.getStringCollection. (Benoy Antony via jing9) - HADOOP-9420. Add percentile or max metric for rpcQueueTime, processing time. - (Liang Xie via wang) - OPTIMIZATIONS HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn) @@ -508,9 +505,6 @@ Release 2.4.0 - UNRELEASED HADOOP-10207. TestUserGroupInformation#testLogin is flaky (jxiang via cmccabe) - HADOOP-10214. Fix multithreaded correctness warnings in ActiveStandbyElector - (Liang Xie via kasha) - Release 2.3.0 - UNRELEASED INCOMPATIBLE CHANGES diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index b12a0fb..3c3da62 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -242,9 +242,4 @@ public static final String HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS = "hadoop.user.group.metrics.percentiles.intervals"; - - public static final String RPC_METRICS_QUANTILE_ENABLE = - "rpc.metrics.quantile.enable"; - public static final String RPC_METRICS_PERCENTILES_INTERVALS_KEY = - "rpc.metrics.percentiles.intervals"; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java index 995d822..fc717c2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java @@ -768,7 +768,7 @@ private void createConnection() throws IOException, KeeperException { } @InterfaceAudience.Private - public synchronized void terminateConnection() { + public void terminateConnection() { if (zkClient == null) { return; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index 9053262..ac79859 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -2193,7 +2193,7 @@ protected Server(String bindAddress, int port, listener = new Listener(); this.port = listener.getAddress().getPort(); connectionManager = new ConnectionManager(); - this.rpcMetrics = RpcMetrics.create(this, conf); + this.rpcMetrics = RpcMetrics.create(this); this.rpcDetailedMetrics = RpcDetailedMetrics.create(this.port); this.tcpNoDelay = conf.getBoolean( CommonConfigurationKeysPublic.IPC_SERVER_TCPNODELAY_KEY, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java index 54b0227..56bfbf7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java @@ -19,17 +19,14 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.metrics2.annotation.Metric; import org.apache.hadoop.metrics2.annotation.Metrics; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.metrics2.lib.MetricsRegistry; import org.apache.hadoop.metrics2.lib.MutableCounterInt; import org.apache.hadoop.metrics2.lib.MutableCounterLong; -import org.apache.hadoop.metrics2.lib.MutableQuantiles; import org.apache.hadoop.metrics2.lib.MutableRate; /** @@ -44,48 +41,26 @@ final Server server; final MetricsRegistry registry; final String name; - final boolean rpcQuantileEnable; - RpcMetrics(Server server, Configuration conf) { + RpcMetrics(Server server) { String port = String.valueOf(server.getListenerAddress().getPort()); - name = "RpcActivityForPort" + port; + name = "RpcActivityForPort"+ port; this.server = server; registry = new MetricsRegistry("rpc").tag("port", "RPC port", port); - int[] intervals = conf.getInts( - CommonConfigurationKeys.RPC_METRICS_PERCENTILES_INTERVALS_KEY); - rpcQuantileEnable = (intervals.length > 0) && conf.getBoolean( - CommonConfigurationKeys.RPC_METRICS_QUANTILE_ENABLE, false); - if (rpcQuantileEnable) { - rpcQueueTimeMillisQuantiles = - new MutableQuantiles[intervals.length]; - rpcProcessingTimeMillisQuantiles = - new MutableQuantiles[intervals.length]; - for (int i = 0; i < intervals.length; i++) { - int interval = intervals[i]; - rpcQueueTimeMillisQuantiles[i] = registry.newQuantiles("rpcQueueTime" - + interval + "s", "rpc queue time in milli second", "ops", - "latency", interval); - rpcProcessingTimeMillisQuantiles[i] = registry.newQuantiles( - "rpcProcessingTime" + interval + "s", - "rpc processing time in milli second", "ops", "latency", interval); - } - } - LOG.debug("Initialized " + registry); + LOG.debug("Initialized "+ registry); } public String name() { return name; } - public static RpcMetrics create(Server server, Configuration conf) { - RpcMetrics m = new RpcMetrics(server, conf); + public static RpcMetrics create(Server server) { + RpcMetrics m = new RpcMetrics(server); return DefaultMetricsSystem.instance().register(m.name, null, m); } @Metric("Number of received bytes") MutableCounterLong receivedBytes; @Metric("Number of sent bytes") MutableCounterLong sentBytes; @Metric("Queue time") MutableRate rpcQueueTime; - MutableQuantiles[] rpcQueueTimeMillisQuantiles; @Metric("Processsing time") MutableRate rpcProcessingTime; - MutableQuantiles[] rpcProcessingTimeMillisQuantiles; @Metric("Number of authentication failures") MutableCounterInt rpcAuthenticationFailures; @Metric("Number of authentication successes") @@ -171,11 +146,6 @@ public void incrReceivedBytes(int count) { //@Override public void addRpcQueueTime(int qTime) { rpcQueueTime.add(qTime); - if (rpcQuantileEnable) { - for (MutableQuantiles q : rpcQueueTimeMillisQuantiles) { - q.add(qTime); - } - } } /** @@ -185,10 +155,5 @@ public void addRpcQueueTime(int qTime) { //@Override public void addRpcProcessingTime(int processingTime) { rpcProcessingTime.add(processingTime); - if (rpcQuantileEnable) { - for (MutableQuantiles q : rpcProcessingTimeMillisQuantiles) { - q.add(processingTime); - } - } } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java index a6e3b1d..33d1f68 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java @@ -20,7 +20,6 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter; import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt; -import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotSame; @@ -68,7 +67,6 @@ import org.apache.hadoop.security.authorize.Service; import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.TokenIdentifier; -import org.apache.hadoop.test.MetricsAsserts; import org.apache.hadoop.test.MockitoUtil; import org.junit.Before; import org.junit.Test; @@ -963,44 +961,6 @@ public void testConnectionPing() throws Exception { } } - @Test - public void testRpcMetrics() throws Exception { - Configuration configuration = new Configuration(); - final int interval = 1; - configuration.setBoolean(CommonConfigurationKeys. - RPC_METRICS_QUANTILE_ENABLE, true); - configuration.set(CommonConfigurationKeys. - RPC_METRICS_PERCENTILES_INTERVALS_KEY, "" + interval); - final Server server = new RPC.Builder(configuration) - .setProtocol(TestProtocol.class).setInstance(new TestImpl()) - .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true) - .build(); - server.start(); - final TestProtocol proxy = RPC.getProxy(TestProtocol.class, - TestProtocol.versionID, server.getListenerAddress(), configuration); - try { - for (int i=0; i<1000; i++) { - proxy.ping(); - proxy.echo("" + i); - } - MetricsRecordBuilder rpcMetrics = - getMetrics(server.getRpcMetrics().name()); - assertTrue("Expected non-zero rpc queue time", - getLongCounter("RpcQueueTimeNumOps", rpcMetrics) > 0); - assertTrue("Expected non-zero rpc processing time", - getLongCounter("RpcProcessingTimeNumOps", rpcMetrics) > 0); - MetricsAsserts.assertQuantileGauges("RpcQueueTime" + interval + "s", - rpcMetrics); - MetricsAsserts.assertQuantileGauges("RpcProcessingTime" + interval + "s", - rpcMetrics); - } finally { - if (proxy != null) { - RPC.stopProxy(proxy); - } - server.stop(); - } - } - public static void main(String[] args) throws IOException { new TestRPC().testCallsInternal(conf); diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 0440ff7..6a5d9ce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -245,9 +245,6 @@ Trunk (Unreleased) HDFS-5715. Use Snapshot ID to indicate the corresponding Snapshot for a FileDiff/DirectoryDiff. (jing9) - HDFS-5721. sharedEditsImage in Namenode#initializeSharedEdits() should be - closed before method returns. (Ted Yu via junping_du) - OPTIMIZATIONS HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe) @@ -739,8 +736,6 @@ Release 2.4.0 - UNRELEASED HDFS-5690. DataNode fails to start in secure mode when dfs.http.policy equals to HTTP_ONLY. (Haohui Mai via jing9) - HDFS-5449. WebHdfs compatibility broken between 2.2 and 1.x / 23.x (kihwal) - BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS HDFS-4985. Add storage type to the protocol and expose it in block report diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 8a9cbd1..f5cda80 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -619,14 +619,8 @@ public static FSNamesystem loadFromDisk(Configuration conf) long loadStart = now(); String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf); - try { - namesystem.loadFSImage(startOpt, fsImage, - HAUtil.isHAEnabled(conf, nameserviceId)); - } catch (IOException ioe) { - LOG.warn("Encountered exception loading fsimage", ioe); - fsImage.close(); - throw ioe; - } + namesystem.loadFSImage(startOpt, fsImage, + HAUtil.isHAEnabled(conf, nameserviceId)); long timeTakenToLoadFSImage = now() - loadStart; LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs"); NameNodeMetrics nnMetrics = NameNode.getNameNodeMetrics(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index 12e26f1..4b2bd66 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -816,20 +816,14 @@ private static boolean format(Configuration conf, boolean force, System.out.println("Formatting using clusterid: " + clusterId); FSImage fsImage = new FSImage(conf, nameDirsToFormat, editDirsToFormat); - try { - FSNamesystem fsn = new FSNamesystem(conf, fsImage); - fsImage.getEditLog().initJournalsForWrite(); - - if (!fsImage.confirmFormat(force, isInteractive)) { - return true; // aborted - } - - fsImage.format(fsn, clusterId); - } catch (IOException ioe) { - LOG.warn("Encountered exception during format: ", ioe); - fsImage.close(); - throw ioe; + FSNamesystem fsn = new FSNamesystem(conf, fsImage); + fsImage.getEditLog().initJournalsForWrite(); + + if (!fsImage.confirmFormat(force, isInteractive)) { + return true; // aborted } + + fsImage.format(fsn, clusterId); return false; } @@ -903,7 +897,6 @@ private static boolean initializeSharedEdits(Configuration conf, } NNStorage existingStorage = null; - FSImage sharedEditsImage = null; try { FSNamesystem fsns = FSNamesystem.loadFromDisk(getConfigurationWithoutSharedEdits(conf)); @@ -913,7 +906,7 @@ private static boolean initializeSharedEdits(Configuration conf, List sharedEditsDirs = FSNamesystem.getSharedEditsDirs(conf); - sharedEditsImage = new FSImage(conf, + FSImage sharedEditsImage = new FSImage(conf, Lists.newArrayList(), sharedEditsDirs); sharedEditsImage.getEditLog().initJournalsForWrite(); @@ -941,13 +934,6 @@ private static boolean initializeSharedEdits(Configuration conf, LOG.error("Could not initialize shared edits dir", ioe); return true; // aborted } finally { - if (sharedEditsImage != null) { - try { - sharedEditsImage.close(); - } catch (IOException ioe) { - LOG.warn("Could not close sharedEditsImage", ioe); - } - } // Have to unlock storage explicitly for the case when we're running in a // unit test, which runs in the same JVM as NNs. if (existingStorage != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java index ac0761d..484eb45 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java @@ -190,29 +190,24 @@ private int doRun() throws IOException { // Load the newly formatted image, using all of the directories (including shared // edits) FSImage image = new FSImage(conf); - try { - image.getStorage().setStorageInfo(storage); - image.initEditLog(); - assert image.getEditLog().isOpenForRead() : + image.getStorage().setStorageInfo(storage); + image.initEditLog(); + assert image.getEditLog().isOpenForRead() : "Expected edit log to be open for read"; + + // Ensure that we have enough edits already in the shared directory to + // start up from the last checkpoint on the active. + if (!checkLogsAvailableForRead(image, imageTxId, curTxId)) { + return ERR_CODE_LOGS_UNAVAILABLE; + } + + image.getStorage().writeTransactionIdFileToStorage(curTxId); - // Ensure that we have enough edits already in the shared directory to - // start up from the last checkpoint on the active. - if (!checkLogsAvailableForRead(image, imageTxId, curTxId)) { - return ERR_CODE_LOGS_UNAVAILABLE; - } - - image.getStorage().writeTransactionIdFileToStorage(curTxId); - - // Download that checkpoint into our storage directories. - MD5Hash hash = TransferFsImage.downloadImageToStorage( + // Download that checkpoint into our storage directories. + MD5Hash hash = TransferFsImage.downloadImageToStorage( otherHttpAddr, imageTxId, storage, true); - image.saveDigestAndRenameCheckpointImage(imageTxId, hash); - } catch (IOException ioe) { - image.close(); - throw ioe; - } + image.saveDigestAndRenameCheckpointImage(imageTxId, hash); return 0; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java index a96f618..63fbda6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java @@ -271,7 +271,7 @@ private static ExtendedBlock toExtendedBlock(final Map m) { } /** Convert a DatanodeInfo to a Json map. */ - static Map toJsonMap(final DatanodeInfo datanodeinfo) { + private static Map toJsonMap(final DatanodeInfo datanodeinfo) { if (datanodeinfo == null) { return null; } @@ -279,9 +279,6 @@ private static ExtendedBlock toExtendedBlock(final Map m) { // TODO: Fix storageID final Map m = new TreeMap(); m.put("ipAddr", datanodeinfo.getIpAddr()); - // 'name' is equivalent to ipAddr:xferPort. Older clients (1.x, 0.23.x) - // expects this instead of the two fields. - m.put("name", datanodeinfo.getXferAddr()); m.put("hostName", datanodeinfo.getHostName()); m.put("storageID", datanodeinfo.getDatanodeUuid()); m.put("xferPort", datanodeinfo.getXferPort()); @@ -328,49 +325,17 @@ private static String getString(Map m, String key, } /** Convert a Json map to an DatanodeInfo object. */ - static DatanodeInfo toDatanodeInfo(final Map m) - throws IOException { + static DatanodeInfo toDatanodeInfo(final Map m) { if (m == null) { return null; } - // ipAddr and xferPort are the critical fields for accessing data. - // If any one of the two is missing, an exception needs to be thrown. - - // Handle the case of old servers (1.x, 0.23.x) sending 'name' instead - // of ipAddr and xferPort. - String ipAddr = getString(m, "ipAddr", null); - int xferPort = getInt(m, "xferPort", -1); - if (ipAddr == null) { - String name = getString(m, "name", null); - if (name != null) { - int colonIdx = name.indexOf(':'); - if (colonIdx > 0) { - ipAddr = name.substring(0, colonIdx); - xferPort = Integer.parseInt(name.substring(colonIdx +1)); - } else { - throw new IOException( - "Invalid value in server response: name=[" + name + "]"); - } - } else { - throw new IOException( - "Missing both 'ipAddr' and 'name' in server response."); - } - // ipAddr is non-null & non-empty string at this point. - } - - // Check the validity of xferPort. - if (xferPort == -1) { - throw new IOException( - "Invalid or missing 'xferPort' in server response."); - } - // TODO: Fix storageID return new DatanodeInfo( - ipAddr, + (String)m.get("ipAddr"), (String)m.get("hostName"), (String)m.get("storageID"), - xferPort, + (int)(long)(Long)m.get("xferPort"), (int)(long)(Long)m.get("infoPort"), getInt(m, "infoSecurePort", 0), (int)(long)(Long)m.get("ipcPort"), @@ -403,8 +368,7 @@ static DatanodeInfo toDatanodeInfo(final Map m) } /** Convert an Object[] to a DatanodeInfo[]. */ - private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) - throws IOException { + private static DatanodeInfo[] toDatanodeInfoArray(final Object[] objects) { if (objects == null) { return null; } else if (objects.length == 0) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java index 6a5acab..f4f7095 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java @@ -24,7 +24,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSUtil; -import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.server.namenode.INodeId; import org.apache.hadoop.util.Time; @@ -62,7 +61,7 @@ public void testHdfsFileStatus() { } @Test - public void testToDatanodeInfoWithoutSecurePort() throws Exception { + public void testToDatanodeInfoWithoutSecurePort() { Map response = new HashMap(); response.put("ipAddr", "127.0.0.1"); @@ -85,63 +84,4 @@ public void testToDatanodeInfoWithoutSecurePort() throws Exception { JsonUtil.toDatanodeInfo(response); } - - @Test - public void testToDatanodeInfoWithName() throws Exception { - Map response = new HashMap(); - - // Older servers (1.x, 0.23, etc.) sends 'name' instead of ipAddr - // and xferPort. - String name = "127.0.0.1:1004"; - response.put("name", name); - response.put("hostName", "localhost"); - response.put("storageID", "fake-id"); - response.put("infoPort", 1338l); - response.put("ipcPort", 1339l); - response.put("capacity", 1024l); - response.put("dfsUsed", 512l); - response.put("remaining", 512l); - response.put("blockPoolUsed", 512l); - response.put("lastUpdate", 0l); - response.put("xceiverCount", 4096l); - response.put("networkLocation", "foo.bar.baz"); - response.put("adminState", "NORMAL"); - response.put("cacheCapacity", 123l); - response.put("cacheUsed", 321l); - - DatanodeInfo di = JsonUtil.toDatanodeInfo(response); - Assert.assertEquals(name, di.getXferAddr()); - - // The encoded result should contain name, ipAddr and xferPort. - Map r = JsonUtil.toJsonMap(di); - Assert.assertEquals(name, (String)r.get("name")); - Assert.assertEquals("127.0.0.1", (String)r.get("ipAddr")); - // In this test, it is Integer instead of Long since json was not actually - // involved in constructing the map. - Assert.assertEquals(1004, (int)(Integer)r.get("xferPort")); - - // Invalid names - String[] badNames = {"127.0.0.1", "127.0.0.1:", ":", "127.0.0.1:sweet", ":123"}; - for (String badName : badNames) { - response.put("name", badName); - checkDecodeFailure(response); - } - - // Missing both name and ipAddr - response.remove("name"); - checkDecodeFailure(response); - - // Only missing xferPort - response.put("ipAddr", "127.0.0.1"); - checkDecodeFailure(response); - } - - private void checkDecodeFailure(Map map) { - try { - JsonUtil.toDatanodeInfo(map); - Assert.fail("Exception not thrown against bad input."); - } catch (Exception e) { - // expected - } - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java index 9c36eeb..5cf2397 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java @@ -182,10 +182,9 @@ public void testSetClasspath() throws IOException { MRApps.setClasspath(environment, job.getConfiguration()); assertTrue(environment.get("CLASSPATH").startsWith( ApplicationConstants.Environment.PWD.$() + File.pathSeparator)); - String yarnAppClasspath = job.getConfiguration().get( - YarnConfiguration.YARN_APPLICATION_CLASSPATH, - StringUtils.join(",", - YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH)); + String yarnAppClasspath = + job.getConfiguration().get( + YarnConfiguration.YARN_APPLICATION_CLASSPATH); if (yarnAppClasspath != null) { yarnAppClasspath = yarnAppClasspath.replaceAll(",\\s*", File.pathSeparator) .trim(); @@ -218,10 +217,7 @@ public void testSetClasspathWithArchives () throws IOException { MRApps.setClasspath(environment, conf); assertTrue(environment.get("CLASSPATH").startsWith( ApplicationConstants.Environment.PWD.$() + File.pathSeparator)); - String confClasspath = job.getConfiguration().get( - YarnConfiguration.YARN_APPLICATION_CLASSPATH, - StringUtils.join(",", - YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH)); + String confClasspath = job.getConfiguration().get(YarnConfiguration.YARN_APPLICATION_CLASSPATH); if (confClasspath != null) { confClasspath = confClasspath.replaceAll(",\\s*", File.pathSeparator) .trim(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java index 74b07c2..fb3e47d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java @@ -44,9 +44,13 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; import org.apache.hadoop.yarn.api.records.NodeState; @@ -371,4 +375,29 @@ public YarnClusterMetrics getYarnClusterMetrics() throws YarnException, IOException { return client.getQueueAclsInfo(); } + + @Override + public ApplicationAttemptReport getApplicationAttemptReport( + ApplicationAttemptId appAttemptId) throws YarnException, IOException { + return client.getApplicationAttemptReport(appAttemptId); + } + + @Override + public List getApplicationAttempts( + ApplicationId appId) throws YarnException, IOException { + return client.getApplicationAttempts(appId); + } + + @Override + public ContainerReport getContainerReport(ContainerId containerId) + throws YarnException, IOException { + return client.getContainerReport(containerId); + } + + @Override + public List getContainers( + ApplicationAttemptId applicationAttemptId) throws YarnException, + IOException { + return client.getContainers(applicationAttemptId); + } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java index 10f8714..6f630ae 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/ssl/TestEncryptedShuffle.java @@ -31,7 +31,6 @@ import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.security.ssl.KeyStoreTestUtil; -import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.junit.After; import org.junit.AfterClass; @@ -87,10 +86,8 @@ private void startCluster(Configuration conf) throws Exception { conf.set("dfs.block.access.token.enable", "false"); conf.set("dfs.permissions", "true"); conf.set("hadoop.security.authentication", "simple"); - String cp = conf.get(YarnConfiguration.YARN_APPLICATION_CLASSPATH, - StringUtils.join(",", - YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH)) - + File.pathSeparator + classpathDir; + String cp = conf.get(YarnConfiguration.YARN_APPLICATION_CLASSPATH) + + File.pathSeparator + classpathDir; conf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, cp); dfsCluster = new MiniDFSCluster(conf, 1, true, null); FileSystem fileSystem = dfsCluster.getFileSystem(); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java index 9a3cb24..b24e20e 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java @@ -860,9 +860,4 @@ public ApplicationResourceUsageReport getAppResourceUsageReport( public List getAppsInQueue(String queue) { return scheduler.getAppsInQueue(queue); } - - @Override - public RMContainer getRMContainer(ContainerId containerId) { - return null; - } } diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt index fa75914..2608837 100644 --- a/hadoop-yarn-project/CHANGES.txt +++ b/hadoop-yarn-project/CHANGES.txt @@ -58,11 +58,6 @@ Release 2.4.0 - UNRELEASED YARN-1029. Added embedded leader election in the ResourceManager. (Karthik Kambatla via vinodkv) - YARN-1490. Introduced the ability to make ResourceManager optionally not kill - all containers when an ApplicationMaster exits. (Jian He via vinodkv) - - YARN-1033. Expose RM active/standby state to Web UI and REST API (kasha) - IMPROVEMENTS YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu) @@ -210,8 +205,6 @@ Release 2.4.0 - UNRELEASED YARN-1568. Rename clusterid to clusterId in ActiveRMInfoProto (kasha) - YARN-1579. ActiveRMInfoProto fields should be optional (kasha) - OPTIMIZATIONS BUG FIXES @@ -402,8 +395,80 @@ Release 2.3.0 - UNRELEASED YARN-1438. Ensure container diagnostics includes exception from container launch. (stevel via acmurthy) - YARN-1138. yarn.application.classpath is set to point to $HADOOP_CONF_DIR - etc., which does not work on Windows. (Chuan Liu via cnauroth) +Branch YARN-321: Generic ApplicationHistoryService + + YARN-930. Bootstrapping ApplicationHistoryService module. (vinodkv) + + YARN-935. Correcting pom.xml to build applicationhistoryserver module + successfully. (Zhijie Shen via vinodkv) + + YARN-962. Fixed bug in application-history proto file and renamed it be just + a client proto file. (Zhijie Shen via vinodkv) + + YARN-947. Implementing the data objects to be used by the History reader + and writer interfaces. (Zhijie Shen via vinodkv) + + YARN-984. Renamed the incorrectly named applicationhistoryservice.records.pb.impl + package to be applicationhistoryservice.records.impl.pb. (Devaraj K via vinodkv) + + YARN-934. Defined a Writer Interface for HistoryStorage. (Zhijie Shen via + vinodkv) + + YARN-925. Defined a Reader Interface for HistoryStorage. (Mayank Bansal via + vinodkv) + + YARN-978. Created ApplicationAttemptReport. (Mayank Bansal via vinodkv) + + YARN-956. Added a testable in-memory HistoryStorage. (Mayank Bansal via + vinodkv) + + YARN-1007. Enhance History Reader interface for Containers. (Mayank Bansal via + devaraj) + + YARN-975. Added a file-system implementation for HistoryStorage. (Zhijie Shen + via vinodkv) + + YARN-1123. Added a new ContainerReport and its Protobuf implementation. (Mayank + Bansal via vinodkv) + + YARN-979. Added more APIs for getting information about ApplicationAttempts + and Containers from ApplicationHistoryProtocol. (Mayank Bansal and Zhijie Shen + via vinodkv) + + YARN-974. Added more information to RMContainer to be collected and recorded in + Application-History. (Zhijie Shen via vinodkv) + + YARN-987. Added ApplicationHistoryManager responsible for exposing reports to + all clients. (Mayank Bansal via vinodkv) + + YARN-953. Changed ResourceManager to start writing history data. (Zhijie Shen + via vinodkv) + + YARN-1266. Implemented PB service and client wrappers for + ApplicationHistoryProtocol. (Mayank Bansal via vinodkv) + + YARN-955. Implemented ApplicationHistoryProtocol handler. (Mayank Bansal via + vinodkv) + + YARN-1242. Changed yarn scripts to be able to start ApplicationHistoryServer + as an individual process. (Mayank Bansal via vinodkv) + + YARN-954. Implemented web UI for the ApplicationHistoryServer and wired it into + the HistoryStorage. (Zhijie Shen via vinodkv) + + YARN-967. Added the client and CLI interfaces for obtaining ApplicationHistory + data. (Mayank Bansal via vinodkv) + + YARN-1023. Added Webservices REST APIs support for Application History. (Zhijie + Shen via vinodkv) + + YARN-1534. Fixed failure of test TestAHSWebApp. (Shinichi Yamashita via vinodkv) + + YARN-1413. Implemented serving of aggregated-logs in the ApplicationHistory + server. (Mayank Bansal via vinodkv) + + YARN-1555. Fixed test failures in applicationhistoryservice.* (Vinod Kumar + Vavilapalli via mayank) Release 2.2.0 - 2013-10-13 diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn index 97c2afe..ac42a9a 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/yarn +++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn @@ -58,10 +58,13 @@ function print_usage(){ echo "where COMMAND is one of:" echo " resourcemanager run the ResourceManager" echo " nodemanager run a nodemanager on each slave" + echo " historyserver run the application history server" echo " rmadmin admin tools" echo " version print the version" echo " jar run a jar file" echo " application prints application(s) report/kill application" + echo " applicationattempt prints applicationattempt(s) report" + echo " container prints container(s) report" echo " node prints node report(s)" echo " logs dump container logs" echo " classpath prints the class path needed to get the" @@ -145,6 +148,10 @@ if [ -d "$HADOOP_YARN_HOME/yarn-server/yarn-server-common/target/classes" ]; the fi if [ -d "$HADOOP_YARN_HOME/yarn-server/yarn-server-resourcemanager/target/classes" ]; then CLASSPATH=${CLASSPATH}:$HADOOP_YARN_HOME/yarn-server/yarn-server-resourcemanager/target/classes + CLASSPATH=${CLASSPATH}:$HADOOP_YARN_HOME/yarn-server/yarn-server-applicationhistoryservice/target/classes +fi +if [ -d "$HADOOP_YARN_HOME/yarn-server/yarn-server-applicationhistoryservice/target/classes" ]; then + CLASSPATH=${CLASSPATH}:$HADOOP_YARN_HOME/yarn-server/yarn-server-applicationhistoryservice/target/classes fi if [ -d "$HADOOP_YARN_HOME/build/test/classes" ]; then CLASSPATH=${CLASSPATH}:$HADOOP_YARN_HOME/target/test/classes @@ -177,9 +184,12 @@ if [ "$COMMAND" = "classpath" ] ; then elif [ "$COMMAND" = "rmadmin" ] ; then CLASS='org.apache.hadoop.yarn.client.cli.RMAdminCLI' YARN_OPTS="$YARN_OPTS $YARN_CLIENT_OPTS" -elif [ "$COMMAND" = "application" ] ; then +elif [ "$COMMAND" = "application" ] || + [ "$COMMAND" = "applicationattempt" ] || + [ "$COMMAND" = "container" ]; then CLASS=org.apache.hadoop.yarn.client.cli.ApplicationCLI YARN_OPTS="$YARN_OPTS $YARN_CLIENT_OPTS" + set -- $COMMAND $@ elif [ "$COMMAND" = "node" ] ; then CLASS=org.apache.hadoop.yarn.client.cli.NodeCLI YARN_OPTS="$YARN_OPTS $YARN_CLIENT_OPTS" @@ -190,6 +200,13 @@ elif [ "$COMMAND" = "resourcemanager" ] ; then if [ "$YARN_RESOURCEMANAGER_HEAPSIZE" != "" ]; then JAVA_HEAP_MAX="-Xmx""$YARN_RESOURCEMANAGER_HEAPSIZE""m" fi +elif [ "$COMMAND" = "historyserver" ] ; then + CLASSPATH=${CLASSPATH}:$YARN_CONF_DIR/ahs-config/log4j.properties + CLASS='org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer' + YARN_OPTS="$YARN_OPTS $YARN_HISTORYSERVER_OPTS" + if [ "$YARN_RESOURCEMANAGER_HEAPSIZE" != "" ]; then + JAVA_HEAP_MAX="-Xmx""$YARN_HISTORYSERVER_HEAPSIZE""m" + fi elif [ "$COMMAND" = "nodemanager" ] ; then CLASSPATH=${CLASSPATH}:$YARN_CONF_DIR/nm-config/log4j.properties CLASS='org.apache.hadoop.yarn.server.nodemanager.NodeManager' diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd index 8afc6ea..121f864 100644 --- a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd +++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd @@ -120,8 +120,11 @@ if "%1" == "--config" ( if exist %HADOOP_YARN_HOME%\yarn-server\yarn-server-resourcemanager\target\classes ( set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\yarn-server\yarn-server-resourcemanager\target\classes + set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\yarn-server\yarn-server-applicationhistoryservice\target\classes + ) + if exist %HADOOP_YARN_HOME%\yarn-server\yarn-server-applicationhistoryservice\target\classes ( + set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\yarn-server\yarn-server-applicationhistoryservice\target\classes ) - if exist %HADOOP_YARN_HOME%\build\test\classes ( set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\build\test\classes ) @@ -138,7 +141,8 @@ if "%1" == "--config" ( goto :eof ) - set yarncommands=resourcemanager nodemanager proxyserver rmadmin version jar application node logs daemonlog + set yarncommands=resourcemanager nodemanager proxyserver rmadmin version jar ^ + application applicationattempt container node logs daemonlog historyserver for %%i in ( %yarncommands% ) do ( if %yarn-command% == %%i set yarncommand=true ) @@ -170,8 +174,21 @@ goto :eof :application set CLASS=org.apache.hadoop.yarn.client.cli.ApplicationCLI set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS% + set yarn-command-arguments=%yarn-command% %yarn-command-arguments% goto :eof +:applicationattempt + set CLASS=org.apache.hadoop.yarn.client.cli.ApplicationCLI + set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS% + set yarn-command-arguments=%yarn-command% %yarn-command-arguments% + goto :eof + +:container + set CLASS=org.apache.hadoop.yarn.client.cli.ApplicationCLI + set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS% + set yarn-command-arguments=%yarn-command% %yarn-command-arguments% + goto :eof + :node set CLASS=org.apache.hadoop.yarn.client.cli.NodeCLI set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS% @@ -186,6 +203,15 @@ goto :eof ) goto :eof +:historyserver + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\ahs-config\log4j.properties + set CLASS=org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer + set YARN_OPTS=%YARN_OPTS% %HADOOP_HISTORYSERVER_OPTS% + if defined YARN_RESOURCEMANAGER_HEAPSIZE ( + set JAVA_HEAP_MAX=-Xmx%YARN_HISTORYSERVER_HEAPSIZE%m + ) + goto :eof + :nodemanager set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\nm-config\log4j.properties set CLASS=org.apache.hadoop.yarn.server.nodemanager.NodeManager @@ -251,10 +277,13 @@ goto :eof @echo where COMMAND is one of: @echo resourcemanager run the ResourceManager @echo nodemanager run a nodemanager on each slave + @echo historyserver run the application history server @echo rmadmin admin tools @echo version print the version @echo jar ^ run a jar file @echo application prints application(s) report/kill application + @echo applicationattempt prints applicationattempt(s) report + @echo container prints container(s) report @echo node prints node report(s) @echo logs dump container logs @echo classpath prints the class path needed to get the diff --git a/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh b/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh index f450740..c2b01be 100644 --- a/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh +++ b/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh @@ -54,6 +54,15 @@ fi # or JAVA_HEAP_MAX with YARN_HEAPMAX as the preferred option of the two. #export YARN_RESOURCEMANAGER_HEAPSIZE=1000 +# Specify the max Heapsize for the HistoryManager using a numerical value +# in the scale of MB. For example, to specify an jvm option of -Xmx1000m, set +# the value to 1000. +# This value will be overridden by an Xmx setting specified in either YARN_OPTS +# and/or YARN_HISTORYSERVER_OPTS. +# If not specified, the default value will be picked from either YARN_HEAPMAX +# or JAVA_HEAP_MAX with YARN_HEAPMAX as the preferred option of the two. +#export YARN_HISTORYSERVER_HEAPSIZE=1000 + # Specify the JVM options to be used when starting the ResourceManager. # These options will be appended to the options specified as YARN_OPTS # and therefore may override any similar flags set in YARN_OPTS diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml index c0df5d8..c019a24 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml @@ -62,6 +62,8 @@ containermanagement_protocol.proto server/yarn_server_resourcemanager_service_protos.proto server/resourcemanager_administration_protocol.proto + application_history_client.proto + server/application_history_server.proto ${project.build.directory}/generated-sources/java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocol.java new file mode 100644 index 0000000..e839063 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocol.java @@ -0,0 +1,277 @@ +/** + * 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.hadoop.yarn.api; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.api.records.Token; +import org.apache.hadoop.yarn.exceptions.YarnException; + +/** + *

The protocol between clients and the ApplicationHistoryServer + * to get the information of completed applications etc.

+ */ +@Public +@Unstable +public interface ApplicationHistoryProtocol { + + /** + *

The interface used by clients to get a report of an Application from + * the ResourceManager.

+ * + *

The client, via {@link GetApplicationReportRequest} provides the + * {@link ApplicationId} of the application.

+ * + *

In secure mode,the ApplicationHistoryServer verifies + * access to the application, queue etc. before accepting the request.

+ * + *

The ApplicationHistoryServer responds with a + * {@link GetApplicationReportResponse} which includes the + * {@link ApplicationReport} for the application.

+ * + *

If the user does not have VIEW_APP access then the + * following fields in the report will be set to stubbed values: + *

+ * + * @param request request for an application report + * @return application report + * @throws YarnException + * @throws IOException + */ + @Public + @Unstable + public GetApplicationReportResponse getApplicationReport( + GetApplicationReportRequest request) throws YarnException, IOException; + + /** + *

The interface used by clients to get a report of all Applications + * in the cluster from the ApplicationHistoryServer.

+ * + *

The ApplicationHistoryServer responds with a + * {@link GetApplicationsResponse} which includes a list of + * {@link ApplicationReport} for all the applications.

+ * + *

If the user does not have VIEW_APP access for an + * application then the corresponding report will be filtered as + * described in {@link #getApplicationReport(GetApplicationReportRequest)}. + *

+ * + * @param request request for reports on all the applications + * @return report on applications matching the given application types + * defined in the request + * @throws YarnException + * @throws IOException + */ + @Public + @Unstable + public GetApplicationsResponse getApplications(GetApplicationsRequest request) + throws YarnException, IOException; + + /** + *

The interface used by clients to get a report of an Application Attempt + * from the ApplicationHistoryServer.

+ * + *

The client, via {@link GetApplicationAttemptReportRequest} provides the + * {@link ApplicationAttemptId} of the application attempt.

+ * + *

In secure mode,the ApplicationHistoryServer verifies access + * to the method before accepting the request.

+ * + *

The ApplicationHistoryServer responds with a + * {@link GetApplicationAttemptReportResponse} which includes the + * {@link ApplicationAttemptReport} for the application attempt.

+ * + *

+ * If the user does not have VIEW_APP access then the following + * fields in the report will be set to stubbed values: + *

+ *

+ * + * @param request + * request for an application attempt report + * @return application attempt report + * @throws YarnException + * @throws IOException + */ + @Public + @Unstable + public GetApplicationAttemptReportResponse getApplicationAttemptReport( + GetApplicationAttemptReportRequest request) throws YarnException, + IOException; + + /** + *

The interface used by clients to get a report of all Application + * attempts in the cluster from the ApplicationHistoryServer.

+ * + *

The ApplicationHistoryServer responds with a + * {@link GetApplicationAttemptsRequest} which includes the + * {@link ApplicationAttemptReport} for all the applications attempts of a + * specified application attempt.

+ * + *

If the user does not have VIEW_APP access for an application + * then the corresponding report will be filtered as described in + * {@link #getApplicationAttemptReport(GetApplicationAttemptReportRequest)}.

+ * + * @param request + * request for reports on all application attempts of an application + * @return reports on all application attempts of an application + * @throws YarnException + * @throws IOException + */ + @Public + @Unstable + public GetApplicationAttemptsResponse getApplicationAttempts( + GetApplicationAttemptsRequest request) throws YarnException, IOException; + + /** + *

The interface used by clients to get a report of an Container from the + * ApplicationHistoryServer.

+ * + *

The client, via {@link GetContainerReportRequest} provides the + * {@link ContainerId} of the container.

+ * + *

In secure mode,the ApplicationHistoryServer verifies access + * to the method before accepting the request.

+ * + *

The ApplicationHistoryServer responds with a + * {@link GetContainerReportResponse} which includes the + * {@link ContainerReport} for the container.

+ * + * @param request + * request for a container report + * @return container report + * @throws YarnException + * @throws IOException + */ + @Public + @Unstable + public GetContainerReportResponse getContainerReport( + GetContainerReportRequest request) throws YarnException, IOException; + + /** + *

The interface used by clients to get a report of Containers for an + * application attempt from the ApplciationHistoryServer.

+ * + *

The client, via {@link GetContainersRequest} provides the + * {@link ApplicationAttemptId} of the application attempt.

+ * + *

In secure mode,the ApplicationHistoryServer verifies access + * to the method before accepting the request.

+ * + *

The ApplicationHistoryServer responds with a + * {@link GetContainersResponse} which includes a list of + * {@link ContainerReport} for all the containers of a specific application + * attempt.

+ * + * @param request + * request for a list of container reports of an application attempt. + * @return reports on all containers of an application attempt + * @throws YarnException + * @throws IOException + */ + @Public + @Unstable + public GetContainersResponse getContainers(GetContainersRequest request) + throws YarnException, IOException; + + /** + *

The interface used by clients to get delegation token, enabling the + * containers to be able to talk to the service using those tokens.

+ * + *

The ApplicationHistoryServer responds with the delegation + * token {@link Token} that can be used by the client to speak to this + * service.

+ * + * @param request request to get a delegation token for the client. + * @return delegation token that can be used to talk to this service + * @throws YarnException + * @throws IOException + */ + @Public + @Unstable + public GetDelegationTokenResponse getDelegationToken( + GetDelegationTokenRequest request) throws YarnException, IOException; + + /** + * Renew an existing delegation token. + * + * @param request the delegation token to be renewed. + * @return the new expiry time for the delegation token. + * @throws YarnException + * @throws IOException + */ + @Private + @Unstable + public RenewDelegationTokenResponse renewDelegationToken( + RenewDelegationTokenRequest request) throws YarnException, IOException; + + /** + * Cancel an existing delegation token. + * + * @param request the delegation token to be cancelled. + * @return an empty response. + * @throws YarnException + * @throws IOException + */ + @Private + @Unstable + public CancelDelegationTokenResponse cancelDelegationToken( + CancelDelegationTokenRequest request) throws YarnException, IOException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptReportRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptReportRequest.java new file mode 100644 index 0000000..495b5e5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptReportRequest.java @@ -0,0 +1,71 @@ +/** + * 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.hadoop.yarn.api.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.util.Records; + +/** + *

The request sent by a client to the ResourceManager to get + * an {@link ApplicationAttemptReport} for an application attempt.

+ * + *

The request should include the {@link ApplicationAttemptId} of the + * application attempt.

+ * + * @see ApplicationAttemptReport + * @see ApplicationHistoryProtocol#getApplicationAttemptReport(GetApplicationAttemptReportRequest) + */ +@Public +@Unstable +public abstract class GetApplicationAttemptReportRequest { + + @Public + @Unstable + public static GetApplicationAttemptReportRequest newInstance( + ApplicationAttemptId applicationAttemptId) { + GetApplicationAttemptReportRequest request = + Records.newRecord(GetApplicationAttemptReportRequest.class); + request.setApplicationAttemptId(applicationAttemptId); + return request; + } + + /** + * Get the ApplicationAttemptId of an application attempt. + * + * @return ApplicationAttemptId of an application attempt + */ + @Public + @Unstable + public abstract ApplicationAttemptId getApplicationAttemptId(); + + /** + * Set the ApplicationAttemptId of an application attempt + * + * @param applicationAttemptId ApplicationAttemptId of an + * application attempt + */ + @Public + @Unstable + public abstract void setApplicationAttemptId( + ApplicationAttemptId applicationAttemptId); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptReportResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptReportResponse.java new file mode 100644 index 0000000..68f1634 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptReportResponse.java @@ -0,0 +1,70 @@ +/** + * 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.hadoop.yarn.api.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.util.Records; + +/** + *

The response sent by the ResourceManager to a client requesting + * an application attempt report.

+ * + *

The response includes an {@link ApplicationAttemptReport} which has the + * details about the particular application attempt

+ * + * @see ApplicationAttemptReport + * @see ApplicationHistoryProtocol#getApplicationAttemptReport(GetApplicationAttemptReportRequest) + */ +@Public +@Unstable +public abstract class GetApplicationAttemptReportResponse { + + @Public + @Unstable + public static GetApplicationAttemptReportResponse newInstance( + ApplicationAttemptReport ApplicationAttemptReport) { + GetApplicationAttemptReportResponse response = + Records.newRecord(GetApplicationAttemptReportResponse.class); + response.setApplicationAttemptReport(ApplicationAttemptReport); + return response; + } + + /** + * Get the ApplicationAttemptReport for the application attempt. + * + * @return ApplicationAttemptReport for the application attempt + */ + @Public + @Unstable + public abstract ApplicationAttemptReport getApplicationAttemptReport(); + + /** + * Get the ApplicationAttemptReport for the application attempt. + * + * @param ApplicationAttemptReport ApplicationAttemptReport for + * the application attempt + */ + @Public + @Unstable + public abstract void setApplicationAttemptReport( + ApplicationAttemptReport applicationAttemptReport); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptsRequest.java new file mode 100644 index 0000000..b632b09 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptsRequest.java @@ -0,0 +1,64 @@ +/** + * 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.hadoop.yarn.api.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.util.Records; + +/** + *

The request from clients to get a list of application attempt reports of + * an application from the ResourceManager.

+ * + * @see ApplicationHistoryProtocol#getApplicationAttempts(GetApplicationAttemptsRequest) + */ +@Public +@Unstable +public abstract class GetApplicationAttemptsRequest { + + @Public + @Unstable + public static GetApplicationAttemptsRequest newInstance( + ApplicationId applicationId) { + GetApplicationAttemptsRequest request = + Records.newRecord(GetApplicationAttemptsRequest.class); + request.setApplicationId(applicationId); + return request; + } + + /** + * Get the ApplicationId of an application + * + * @return ApplicationId of an application + */ + @Public + @Unstable + public abstract ApplicationId getApplicationId(); + + /** + * Set the ApplicationId of an application + * + * @param ApplicationId ApplicationId of an application + */ + @Public + @Unstable + public abstract void setApplicationId(ApplicationId applicationId); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptsResponse.java new file mode 100644 index 0000000..5524ea4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetApplicationAttemptsResponse.java @@ -0,0 +1,73 @@ +/** + * 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.hadoop.yarn.api.protocolrecords; + +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.util.Records; + +/** + *

The response sent by the ResourceManager to a client + * requesting a list of {@link ApplicationAttemptReport} for application + * attempts.

+ * + *

The ApplicationAttemptReport for each application includes + * the details of an application attempt.

+ * + * @see ApplicationAttemptReport + * @see ApplicationHistoryProtocol#getApplicationAttempts(GetApplicationAttemptsRequest) + */ +@Public +@Unstable +public abstract class GetApplicationAttemptsResponse { + + @Public + @Unstable + public static GetApplicationAttemptsResponse newInstance( + List applicationAttempts) { + GetApplicationAttemptsResponse response = + Records.newRecord(GetApplicationAttemptsResponse.class); + response.setApplicationAttemptList(applicationAttempts); + return response; + } + + /** + * Get a list of ApplicationReport of an application. + * + * @return a list of ApplicationReport of an application + */ + @Public + @Unstable + public abstract List getApplicationAttemptList(); + + /** + * Get a list of ApplicationReport of an application. + * + * @param applicationAttempts a list of ApplicationReport of an + * application + */ + @Public + @Unstable + public abstract void setApplicationAttemptList( + List applicationAttempts); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerReportRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerReportRequest.java new file mode 100644 index 0000000..6e6ce55 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerReportRequest.java @@ -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.hadoop.yarn.api.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.util.Records; + +/** + *

The request sent by a client to the ResourceManager to + * get an {@link ContainerReport} for a container.

+ */ +@Public +@Unstable +public abstract class GetContainerReportRequest { + + @Public + @Unstable + public static GetContainerReportRequest newInstance(ContainerId containerId) { + GetContainerReportRequest request = Records.newRecord(GetContainerReportRequest.class); + request.setContainerId(containerId); + return request; + } + + /** + * Get the ContainerId of the Container. + * + * @return ContainerId of the Container + */ + @Public + @Unstable + public abstract ContainerId getContainerId(); + + /** + * Set the ContainerId of the container + * + * @param ContainerId + * ContainerId of the container + */ + @Public + @Unstable + public abstract void setContainerId(ContainerId containerId); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerReportResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerReportResponse.java new file mode 100644 index 0000000..46b5cec --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerReportResponse.java @@ -0,0 +1,59 @@ +/** + * 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.hadoop.yarn.api.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.util.Records; + +/** + *

The response sent by the ResourceManager to a client + * requesting a container report.

+ * + *

The response includes a {@link ContainerReport} which has details of a + * container.

+ * + */ +@Public +@Unstable +public abstract class GetContainerReportResponse { + @Public + @Unstable + public static GetContainerReportResponse newInstance( + ContainerReport containerReport) { + GetContainerReportResponse response = + Records.newRecord(GetContainerReportResponse.class); + response.setContainerReport(containerReport); + return response; + } + + /** + * Get the ContainerReport for the container. + * + * @return ContainerReport for the container + */ + @Public + @Unstable + public abstract ContainerReport getContainerReport(); + + @Public + @Unstable + public abstract void setContainerReport(ContainerReport containerReport); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainersRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainersRequest.java new file mode 100644 index 0000000..2f62d9d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainersRequest.java @@ -0,0 +1,65 @@ +/** + * 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.hadoop.yarn.api.protocolrecords; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.util.Records; + +/** + *

The request from clients to get a list of container reports, which belong + * to an application attempt from the ResourceManager.

+ * + * @see ApplicationHistoryProtocol#getContainers(GetContainersRequest) + */ +@Public +@Unstable +public abstract class GetContainersRequest { + @Public + @Unstable + public static GetContainersRequest newInstance( + ApplicationAttemptId applicationAttemptId) { + GetContainersRequest request = + Records.newRecord(GetContainersRequest.class); + request.setApplicationAttemptId(applicationAttemptId); + return request; + } + + /** + * Get the ApplicationAttemptId of an application attempt. + * + * @return ApplicationAttemptId of an application attempt + */ + @Public + @Unstable + public abstract ApplicationAttemptId getApplicationAttemptId(); + + /** + * Set the ApplicationAttemptId of an application attempt + * + * @param applicationAttemptId ApplicationAttemptId of an + * application attempt + */ + @Public + @Unstable + public abstract void setApplicationAttemptId( + ApplicationAttemptId applicationAttemptId); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainersResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainersResponse.java new file mode 100644 index 0000000..4256bae --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainersResponse.java @@ -0,0 +1,75 @@ +/** + * 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.hadoop.yarn.api.protocolrecords; + +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.util.Records; + +/** + *

The response sent by the ResourceManager to a client + * requesting a list of {@link ContainerReport} for containers.

+ * + *

The ContainerReport for each container includes the container + * details.

+ * + * @see ContainerReport + * @see ApplicationHistoryProtocol#getContainers(GetContainersRequest) + */ +@Public +@Unstable +public abstract class GetContainersResponse { + + @Public + @Unstable + public static GetContainersResponse newInstance(List containers) { + GetContainersResponse response = + Records.newRecord(GetContainersResponse.class); + response.setContainerList(containers); + return response; + } + + /** + * Get a list of ContainerReport for all the containers of an + * application attempt. + * + * @return a list of ContainerReport for all the containers of + * an application attempt + * + */ + @Public + @Unstable + public abstract List getContainerList(); + + /** + * Set a list of ContainerReport for all the containers of an + * application attempt. + * + * @param containers a list of ContainerReport for all the + * containers of an application attempt + * + */ + @Public + @Unstable + public abstract void setContainerList(List containers); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java new file mode 100644 index 0000000..d20c10b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptReport.java @@ -0,0 +1,165 @@ +/** + * 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.hadoop.yarn.api.records; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.util.Records; + +/** + *

+ * ApplicationAttemptReport is a report of an application attempt. + *

+ * + *

+ * It includes details such as: + *

    + *
  • {@link ApplicationAttemptId} of the application.
  • + *
  • Host on which the ApplicationMaster of this attempt is + * running.
  • + *
  • RPC port of the ApplicationMaster of this attempt.
  • + *
  • Tracking URL.
  • + *
  • Diagnostic information in case of errors.
  • + *
  • {@link YarnApplicationAttemptState} of the application attempt.
  • + *
  • {@link ContainerId} of the master Container.
  • + *
+ *

+ * + */ +@Public +@Unstable +public abstract class ApplicationAttemptReport { + + @Private + @Unstable + public static ApplicationAttemptReport newInstance( + ApplicationAttemptId applicationAttemptId, String host, int rpcPort, + String url, String diagnostics, YarnApplicationAttemptState state, + ContainerId amContainerId) { + ApplicationAttemptReport report = + Records.newRecord(ApplicationAttemptReport.class); + report.setApplicationAttemptId(applicationAttemptId); + report.setHost(host); + report.setRpcPort(rpcPort); + report.setTrackingUrl(url); + report.setDiagnostics(diagnostics); + report.setYarnApplicationAttemptState(state); + report.setAMContainerId(amContainerId); + return report; + } + + /** + * Get the YarnApplicationAttemptState of the application attempt. + * + * @return YarnApplicationAttemptState of the application attempt + */ + @Public + @Unstable + public abstract YarnApplicationAttemptState getYarnApplicationAttemptState(); + + @Private + @Unstable + public abstract void setYarnApplicationAttemptState( + YarnApplicationAttemptState yarnApplicationAttemptState); + + /** + * Get the RPC port of this attempt ApplicationMaster. + * + * @return RPC port of this attempt ApplicationMaster + */ + @Public + @Unstable + public abstract int getRpcPort(); + + @Private + @Unstable + public abstract void setRpcPort(int rpcPort); + + /** + * Get the host on which this attempt of + * ApplicationMaster is running. + * + * @return host on which this attempt of + * ApplicationMaster is running + */ + @Public + @Unstable + public abstract String getHost(); + + @Private + @Unstable + public abstract void setHost(String host); + + /** + * Get the diagnositic information of the application attempt in case + * of errors. + * + * @return diagnositic information of the application attempt in case + * of errors + */ + @Public + @Unstable + public abstract String getDiagnostics(); + + @Private + @Unstable + public abstract void setDiagnostics(String diagnostics); + + /** + * Get the tracking url for the application attempt. + * + * @return tracking url for the application attempt + */ + @Public + @Unstable + public abstract String getTrackingUrl(); + + @Private + @Unstable + public abstract void setTrackingUrl(String url); + + /** + * Get the ApplicationAttemptId of this attempt of the + * application + * + * @return ApplicationAttemptId of the attempt + */ + @Public + @Unstable + public abstract ApplicationAttemptId getApplicationAttemptId(); + + @Private + @Unstable + public abstract void setApplicationAttemptId( + ApplicationAttemptId applicationAttemptId); + + /** + * Get the ContainerId of AMContainer for this attempt + * + * @return ContainerId of the attempt + */ + @Public + @Unstable + public abstract ContainerId getAMContainerId(); + + @Private + @Unstable + public abstract void setAMContainerId(ContainerId amContainerId); +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java index a320bb2..eaea13e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java @@ -24,7 +24,6 @@ import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; -import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.util.Records; /** @@ -58,8 +57,7 @@ public static ApplicationSubmissionContext newInstance( ApplicationId applicationId, String applicationName, String queue, Priority priority, ContainerLaunchContext amContainer, boolean isUnmanagedAM, boolean cancelTokensWhenComplete, - int maxAppAttempts, Resource resource, String applicationType, - boolean keepContainers) { + int maxAppAttempts, Resource resource, String applicationType) { ApplicationSubmissionContext context = Records.newRecord(ApplicationSubmissionContext.class); context.setApplicationId(applicationId); @@ -72,7 +70,6 @@ public static ApplicationSubmissionContext newInstance( context.setMaxAppAttempts(maxAppAttempts); context.setResource(resource); context.setApplicationType(applicationType); - context.setKeepContainersAcrossApplicationAttempts(keepContainers); return context; } @@ -82,18 +79,6 @@ public static ApplicationSubmissionContext newInstance( ApplicationId applicationId, String applicationName, String queue, Priority priority, ContainerLaunchContext amContainer, boolean isUnmanagedAM, boolean cancelTokensWhenComplete, - int maxAppAttempts, Resource resource, String applicationType) { - return newInstance(applicationId, applicationName, queue, priority, - amContainer, isUnmanagedAM, cancelTokensWhenComplete, maxAppAttempts, - resource, null, false); - } - - @Public - @Stable - public static ApplicationSubmissionContext newInstance( - ApplicationId applicationId, String applicationName, String queue, - Priority priority, ContainerLaunchContext amContainer, - boolean isUnmanagedAM, boolean cancelTokensWhenComplete, int maxAppAttempts, Resource resource) { return newInstance(applicationId, applicationName, queue, priority, amContainer, isUnmanagedAM, cancelTokensWhenComplete, maxAppAttempts, @@ -283,35 +268,4 @@ public static ApplicationSubmissionContext newInstance( @Public @Stable public abstract void setApplicationType(String applicationType); - - - /** - * Get the flag which indicates whether to keep containers across application - * attempts or not. - * - * @return the flag which indicates whether to keep containers across - * application attempts or not. - */ - @Public - @Stable - public abstract boolean getKeepContainersAcrossApplicationAttempts(); - - /** - * Set the flag which indicates whether to keep containers across application - * attempts. - *

- * If the flag is true, running containers will not be killed when application - * attempt fails and these containers will be retrieved by the new application - * attempt on registration via - * {@link ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)}. - *

- * - * @param keepContainers - * the flag which indicates whether to keep containers across - * application attempts. - */ - @Public - @Stable - public abstract void setKeepContainersAcrossApplicationAttempts( - boolean keepContainers); } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java index 73e8085..ccbe6f1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java @@ -46,20 +46,10 @@ public static ContainerId newInstance(ApplicationAttemptId appAttemptId, } /** - * Get the ApplicationAttemptId of the application to which the - * Container was assigned. - *

- * Note: If containers are kept alive across application attempts via - * {@link ApplicationSubmissionContext#setKeepContainersAcrossApplicationAttempts(boolean)} - * the ContainerId does not necessarily contain the current - * running application attempt's ApplicationAttemptId This - * container can be allocated by previously exited application attempt and - * managed by the current running attempt thus have the previous application - * attempt's ApplicationAttemptId. - *

- * - * @return ApplicationAttemptId of the application to which the - * Container was assigned + * Get the ApplicationAttemptId of the application to which + * the Container was assigned. + * @return ApplicationAttemptId of the application to which + * the Container was assigned */ @Public @Stable diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java new file mode 100644 index 0000000..7513f16 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java @@ -0,0 +1,202 @@ +/** + * 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.hadoop.yarn.api.records; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.util.Records; + +/** + *

+ * ContainerReport is a report of an container. + *

+ * + *

+ * It includes details such as: + *

    + *
  • {@link ContainerId} of the container.
  • + *
  • Allocated Resources to the container.
  • + *
  • Assigned Node id.
  • + *
  • Assigned Priority.
  • + *
  • Start Time.
  • + *
  • Finish Time.
  • + *
  • Container Exit Status.
  • + *
  • {@link ContainerState} of the container.
  • + *
  • Diagnostic information in case of errors.
  • + *
  • Log URL.
  • + *
+ *

+ * + */ + +@Public +@Unstable +public abstract class ContainerReport { + @Private + @Unstable + public static ContainerReport newInstance(ContainerId containerId, + Resource allocatedResource, NodeId assignedNode, Priority priority, + long startTime, long finishTime, String diagnosticInfo, String logUrl, + int containerExitStatus, ContainerState containerState) { + ContainerReport report = Records.newRecord(ContainerReport.class); + report.setContainerId(containerId); + report.setAllocatedResource(allocatedResource); + report.setAssignedNode(assignedNode); + report.setPriority(priority); + report.setStartTime(startTime); + report.setFinishTime(finishTime); + report.setDiagnosticsInfo(diagnosticInfo); + report.setLogUrl(logUrl); + report.setContainerExitStatus(containerExitStatus); + report.setContainerState(containerState); + return report; + } + + /** + * Get the ContainerId of the container. + * + * @return ContainerId of the container. + */ + @Public + @Unstable + public abstract ContainerId getContainerId(); + + @Public + @Unstable + public abstract void setContainerId(ContainerId containerId); + + /** + * Get the allocated Resource of the container. + * + * @return allocated Resource of the container. + */ + @Public + @Unstable + public abstract Resource getAllocatedResource(); + + @Public + @Unstable + public abstract void setAllocatedResource(Resource resource); + + /** + * Get the allocated NodeId where container is running. + * + * @return allocated NodeId where container is running. + */ + @Public + @Unstable + public abstract NodeId getAssignedNode(); + + @Public + @Unstable + public abstract void setAssignedNode(NodeId nodeId); + + /** + * Get the allocated Priority of the container. + * + * @return allocated Priority of the container. + */ + @Public + @Unstable + public abstract Priority getPriority(); + + @Public + @Unstable + public abstract void setPriority(Priority priority); + + /** + * Get the Start time of the container. + * + * @return Start time of the container + */ + @Public + @Unstable + public abstract long getStartTime(); + + @Public + @Unstable + public abstract void setStartTime(long startTime); + + /** + * Get the Finish time of the container. + * + * @return Finish time of the container + */ + @Public + @Unstable + public abstract long getFinishTime(); + + @Public + @Unstable + public abstract void setFinishTime(long finishTime); + + /** + * Get the DiagnosticsInfo of the container. + * + * @return DiagnosticsInfo of the container + */ + @Public + @Unstable + public abstract String getDiagnosticsInfo(); + + @Public + @Unstable + public abstract void setDiagnosticsInfo(String diagnosticsInfo); + + /** + * Get the LogURL of the container. + * + * @return LogURL of the container + */ + @Public + @Unstable + public abstract String getLogUrl(); + + @Public + @Unstable + public abstract void setLogUrl(String logUrl); + + /** + * Get the final ContainerState of the container. + * + * @return final ContainerState of the container. + */ + @Public + @Unstable + public abstract ContainerState getContainerState(); + + @Public + @Unstable + public abstract void setContainerState(ContainerState containerState); + + /** + * Get the final exit status of the container. + * + * @return final exit status of the container. + */ + @Public + @Unstable + public abstract int getContainerExitStatus(); + + @Public + @Unstable + public abstract void setContainerExitStatus(int containerExitStatus); + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationAttemptState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationAttemptState.java new file mode 100644 index 0000000..2049a34 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationAttemptState.java @@ -0,0 +1,66 @@ +/** + * 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.hadoop.yarn.api.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Stable; + +/** + * Enumeration of various states of a RMAppAttempt. + */ +@Public +@Stable +public enum YarnApplicationAttemptState { + /** AppAttempt was just created. */ + NEW, + + /** AppAttempt has been submitted. */ + SUBMITTED, + + /**AppAttempt was scheduled*/ + SCHEDULED, + + /**Acquired AM Container from Scheduler and Saving AppAttempt Data*/ + ALLOCATED_SAVING, + + /**AppAttempt Data was saved*/ + ALLOCATED, + + /**AppAttempt was launched*/ + LAUNCHED, + + /** AppAttempt failed. */ + FAILED, + + /** AppAttempt is currently running. */ + RUNNING, + + /** AppAttempt is waiting for state bing saved*/ + FINAL_SAVING, + + /** AppAttempt is finishing. */ + FINISHING, + + /** AppAttempt finished successfully. */ + FINISHED, + + /** AppAttempt was terminated by a user or admin. */ + KILLED + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 4adba79..4a4a750 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -263,6 +263,22 @@ RM_PREFIX + "nodemanagers.heartbeat-interval-ms"; public static final long DEFAULT_RM_NM_HEARTBEAT_INTERVAL_MS = 1000; + /** The setting that controls whether RM writes history data. */ + public static final String RM_HISTORY_WRITER_ENABLED = RM_PREFIX + + "history-writer.enabled"; + public static final boolean DEFAULT_RM_HISTORY_WRITER_ENABLED = false; + + /** Number of worker threads that write the history data. */ + public static final String RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE = + RM_PREFIX + "history-writer.multi-threaded-dispatcher.pool-size"; + public static final int DEFAULT_RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE = + 10; + + /** The implementation class of ApplicationHistoryStore, which is to be used + * by RMApplicationHistoryWriter. */ + public static final String RM_HISTORY_WRITER_CLASS = RM_PREFIX + + "history-writer.class"; + //Delegation token related keys public static final String DELEGATION_KEY_UPDATE_INTERVAL_KEY = RM_PREFIX + "delegation.key.update-interval"; @@ -929,6 +945,58 @@ YARN_PREFIX + "app.container.log.filesize"; //////////////////////////////// + // AHS Configs + //////////////////////////////// + + public static final String AHS_PREFIX = YARN_PREFIX + "ahs."; + + /** URI for FileSystemApplicationHistoryStore */ + public static final String FS_HISTORY_STORE_URI = AHS_PREFIX + "fs-history-store.uri"; + + /** T-file compression types used to compress history data.*/ + public static final String FS_HISTORY_STORE_COMPRESSION_TYPE = AHS_PREFIX + "fs-history-store.compression-type"; + public static final String DEFAULT_FS_HISTORY_STORE_COMPRESSION_TYPE = "none"; + + /** AHS store class */ + public static final String AHS_STORE = AHS_PREFIX + "store.class"; + + /** host:port address for Application History Server API. */ + public static final String AHS_ADDRESS = AHS_PREFIX + "address"; + public static final int DEFAULT_AHS_PORT = 10200; + public static final String DEFAULT_AHS_ADDRESS = "0.0.0.0:" + + DEFAULT_AHS_PORT; + + /** The number of threads to handle client API requests. */ + public static final String AHS_CLIENT_THREAD_COUNT = AHS_PREFIX + + "client.thread-count"; + public static final int DEFAULT_AHS_CLIENT_THREAD_COUNT = 10; + + + /** The address of the AHS web application.*/ + public static final String AHS_WEBAPP_ADDRESS = AHS_PREFIX + + "webapp.address"; + + public static final int DEFAULT_AHS_WEBAPP_PORT = 8188; + public static final String DEFAULT_AHS_WEBAPP_ADDRESS = "0.0.0.0:" + + DEFAULT_AHS_WEBAPP_PORT; + + /** The https address of the AHS web application.*/ + public static final String AHS_WEBAPP_HTTPS_ADDRESS = AHS_PREFIX + + "webapp.https.address"; + + public static final int DEFAULT_AHS_WEBAPP_HTTPS_PORT = 8190; + public static final String DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS = "0.0.0.0:" + + DEFAULT_AHS_WEBAPP_HTTPS_PORT; + + /**The kerberos principal to be used for spnego filter for AHS.*/ + public static final String AHS_WEBAPP_SPNEGO_USER_NAME_KEY = + AHS_PREFIX + "webapp.spnego-principal"; + + /**The kerberos keytab to be used for spnego filter for AHS.*/ + public static final String AHS_WEBAPP_SPNEGO_KEYTAB_FILE_KEY = + AHS_PREFIX + "webapp.spnego-keytab-file"; + + //////////////////////////////// // Other Configs //////////////////////////////// diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationAttemptNotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationAttemptNotFoundException.java new file mode 100644 index 0000000..e54f909 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationAttemptNotFoundException.java @@ -0,0 +1,49 @@ +/** + * 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.hadoop.yarn.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; + +/** + * This exception is thrown on + * {@link ApplicationHistoryProtocol#getApplicationAttemptReport + * (GetApplicationAttemptReportRequest)} + * API when the Application Attempt doesn't exist in Application History Server + */ +@Public +@Unstable +public class ApplicationAttemptNotFoundException extends YarnException { + + private static final long serialVersionUID = 8694508L; + + public ApplicationAttemptNotFoundException(Throwable cause) { + super(cause); + } + + public ApplicationAttemptNotFoundException(String message) { + super(message); + } + + public ApplicationAttemptNotFoundException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationNotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationNotFoundException.java index 8f9a9e2..da83c39 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationNotFoundException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationNotFoundException.java @@ -18,14 +18,19 @@ package org.apache.hadoop.yarn.exceptions; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; /** * This exception is thrown on - * {@link ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest)} API - * when the Application doesn't exist in RM + * {@link ApplicationClientProtocol#getApplicationReport + * (GetApplicationReportRequest)} API + * when the Application doesn't exist in RM and AHS */ +@Public +@Unstable public class ApplicationNotFoundException extends YarnException{ private static final long serialVersionUID = 8694408L; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ContainerNotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ContainerNotFoundException.java new file mode 100644 index 0000000..aba467a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ContainerNotFoundException.java @@ -0,0 +1,49 @@ +/** + * 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.hadoop.yarn.exceptions; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; + +/** + * This exception is thrown on + * {@link ApplicationHistoryProtocol#getContainerReport + * (GetContainerReportRequest)} + * API when the container doesn't exist in AHS + */ +@Public +@Unstable +public class ContainerNotFoundException extends YarnException { + + private static final long serialVersionUID = 8694608L; + + public ContainerNotFoundException(Throwable cause) { + super(cause); + } + + public ContainerNotFoundException(String message) { + super(message); + } + + public ContainerNotFoundException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/application_history_client.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/application_history_client.proto new file mode 100644 index 0000000..7ad06c9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/application_history_client.proto @@ -0,0 +1,39 @@ +/** + * 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. + */ + +option java_package = "org.apache.hadoop.yarn.proto"; +option java_outer_classname = "ApplicationHistoryProtocol"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +package hadoop.yarn; + +import "Security.proto"; +import "yarn_service_protos.proto"; + +service ApplicationHistoryProtocolService { + rpc getApplicationReport (GetApplicationReportRequestProto) returns (GetApplicationReportResponseProto); + rpc getApplications (GetApplicationsRequestProto) returns (GetApplicationsResponseProto); + rpc getApplicationAttemptReport (GetApplicationAttemptReportRequestProto) returns (GetApplicationAttemptReportResponseProto); + rpc getApplicationAttempts (GetApplicationAttemptsRequestProto) returns (GetApplicationAttemptsResponseProto); + rpc getContainerReport (GetContainerReportRequestProto) returns (GetContainerReportResponseProto); + rpc getContainers (GetContainersRequestProto) returns (GetContainersResponseProto); + rpc getDelegationToken(hadoop.common.GetDelegationTokenRequestProto) returns (hadoop.common.GetDelegationTokenResponseProto); + rpc renewDelegationToken(hadoop.common.RenewDelegationTokenRequestProto) returns (hadoop.common.RenewDelegationTokenResponseProto); + rpc cancelDelegationToken(hadoop.common.CancelDelegationTokenRequestProto) returns (hadoop.common.CancelDelegationTokenResponseProto); +} + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/application_history_server.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/application_history_server.proto new file mode 100644 index 0000000..64e7af8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/application_history_server.proto @@ -0,0 +1,113 @@ +/** + * 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. + */ + +option java_package = "org.apache.hadoop.yarn.proto"; +option java_outer_classname = "ApplicationHistoryServerProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +package hadoop.yarn; + +import "yarn_protos.proto"; + +message ApplicationHistoryDataProto { + optional ApplicationIdProto application_id = 1; + optional string application_name = 2; + optional string application_type = 3; + optional string user = 4; + optional string queue = 5; + optional int64 submit_time = 6; + optional int64 start_time = 7; + optional int64 finish_time = 8; + optional string diagnostics_info = 9; + optional FinalApplicationStatusProto final_application_status = 10; + optional YarnApplicationStateProto yarn_application_state = 11; +} + +message ApplicationStartDataProto { + optional ApplicationIdProto application_id = 1; + optional string application_name = 2; + optional string application_type = 3; + optional string user = 4; + optional string queue = 5; + optional int64 submit_time = 6; + optional int64 start_time = 7; +} + +message ApplicationFinishDataProto { + optional ApplicationIdProto application_id = 1; + optional int64 finish_time = 2; + optional string diagnostics_info = 3; + optional FinalApplicationStatusProto final_application_status = 4; + optional YarnApplicationStateProto yarn_application_state = 5; +} + +message ApplicationAttemptHistoryDataProto { + optional ApplicationAttemptIdProto application_attempt_id = 1; + optional string host = 2; + optional int32 rpc_port = 3; + optional string tracking_url = 4; + optional string diagnostics_info = 5; + optional FinalApplicationStatusProto final_application_status = 6; + optional ContainerIdProto master_container_id = 7; + optional YarnApplicationAttemptStateProto yarn_application_attempt_state = 8; +} + +message ApplicationAttemptStartDataProto { + optional ApplicationAttemptIdProto application_attempt_id = 1; + optional string host = 2; + optional int32 rpc_port = 3; + optional ContainerIdProto master_container_id = 4; +} + +message ApplicationAttemptFinishDataProto { + optional ApplicationAttemptIdProto application_attempt_id = 1; + optional string tracking_url = 2; + optional string diagnostics_info = 3; + optional FinalApplicationStatusProto final_application_status = 4; + optional YarnApplicationAttemptStateProto yarn_application_attempt_state = 5; +} + +message ContainerHistoryDataProto { + optional ContainerIdProto container_id = 1; + optional ResourceProto allocated_resource = 2; + optional NodeIdProto assigned_node_id = 3; + optional PriorityProto priority = 4; + optional int64 start_time = 5; + optional int64 finish_time = 6; + optional string diagnostics_info = 7; + optional string log_url = 8; + optional int32 container_exit_status = 9; + optional ContainerStateProto container_state = 10; +} + +message ContainerStartDataProto { + optional ContainerIdProto container_id = 1; + optional ResourceProto allocated_resource = 2; + optional NodeIdProto assigned_node_id = 3; + optional PriorityProto priority = 4; + optional int64 start_time = 5; +} + +message ContainerFinishDataProto { + optional ContainerIdProto container_id = 1; + optional int64 finish_time = 2; + optional string diagnostics_info = 3; + optional string log_url = 4; + optional int32 container_exit_status = 5; + optional ContainerStateProto container_state = 6; +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto index aa9e0c6..e27f12a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto @@ -138,6 +138,6 @@ message RMStateVersionProto { ///////////// RM Failover related records //////////////////////// ////////////////////////////////////////////////////////////////// message ActiveRMInfoProto { - optional string clusterId = 1; - optional string rmId = 2; + required string clusterId = 1; + required string rmId = 2; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto index c90c263..6e01df4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto @@ -87,6 +87,19 @@ message ContainerProto { optional hadoop.common.TokenProto container_token = 6; } +message ContainerReportProto { + optional ContainerIdProto container_id = 1; + optional ResourceProto resource = 2; + optional NodeIdProto node_id = 3; + optional PriorityProto priority = 4; + optional int64 start_time = 5; + optional int64 finish_time = 6; + optional string diagnostics_info = 7 [default = "N/A"]; + optional string log_url = 8; + optional int32 container_exit_status = 9; + optional ContainerStateProto container_state = 10; +} + enum YarnApplicationStateProto { NEW = 1; NEW_SAVING = 2; @@ -98,6 +111,21 @@ enum YarnApplicationStateProto { KILLED = 8; } +enum YarnApplicationAttemptStateProto { + APP_ATTEMPT_NEW = 1; + APP_ATTEMPT_SUBMITTED = 2; + APP_ATTEMPT_SCHEDULED = 3; + APP_ATTEMPT_ALLOCATED_SAVING = 4; + APP_ATTEMPT_ALLOCATED = 5; + APP_ATTEMPT_LAUNCHED = 6; + APP_ATTEMPT_FAILED = 7; + APP_ATTEMPT_RUNNING = 8; + APP_ATTEMPT_FINAL_SAVING = 9; + APP_ATTEMPT_FINISHING = 10; + APP_ATTEMPT_FINISHED = 11; + APP_ATTEMPT_KILLED = 12; +} + enum FinalApplicationStatusProto { APP_UNDEFINED = 0; APP_SUCCEEDED = 1; @@ -164,6 +192,16 @@ message ApplicationReportProto { optional hadoop.common.TokenProto am_rm_token = 19; } +message ApplicationAttemptReportProto { + optional ApplicationAttemptIdProto application_attempt_id = 1; + optional string host = 2; + optional int32 rpc_port = 3; + optional string tracking_url = 4; + optional string diagnostics = 5 [default = "N/A"]; + optional YarnApplicationAttemptStateProto yarn_application_attempt_state = 6; + optional ContainerIdProto am_container_id = 7; +} + enum NodeStateProto { NS_NEW = 1; NS_RUNNING = 2; @@ -248,7 +286,6 @@ message ApplicationSubmissionContextProto { optional int32 maxAppAttempts = 8 [default = 0]; optional ResourceProto resource = 9; optional string applicationType = 10 [default = "YARN"]; - optional bool keep_containers_across_application_attempts = 11 [default = false]; } enum ApplicationAccessTypeProto { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto index a4631d1..5267ce2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto @@ -230,3 +230,39 @@ message GetContainerStatusesResponseProto { repeated ContainerStatusProto status = 1; repeated ContainerExceptionMapProto failed_requests = 2; } + +////////////////////////////////////////////////////// +/////// Application_History_Protocol ///////////////// +////////////////////////////////////////////////////// + +message GetApplicationAttemptReportRequestProto { + optional ApplicationAttemptIdProto application_attempt_id = 1; +} + +message GetApplicationAttemptReportResponseProto { + optional ApplicationAttemptReportProto application_attempt_report = 1; +} + +message GetApplicationAttemptsRequestProto { + optional ApplicationIdProto application_id = 1; +} + +message GetApplicationAttemptsResponseProto { + repeated ApplicationAttemptReportProto application_attempts = 1; +} + +message GetContainerReportRequestProto { + optional ContainerIdProto container_id = 1; +} + +message GetContainerReportResponseProto { + optional ContainerReportProto container_report = 1; +} + +message GetContainersRequestProto { + optional ApplicationAttemptIdProto application_attempt_id = 1; +} + +message GetContainersResponseProto { + repeated ContainerReportProto containers = 1; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java new file mode 100644 index 0000000..ebfd95a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AHSClient.java @@ -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.hadoop.yarn.client.api; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.client.api.impl.AHSClientImpl; +import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException; +import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException; +import org.apache.hadoop.yarn.exceptions.YarnException; + +@InterfaceAudience.Public +@InterfaceStability.Stable +public abstract class AHSClient extends AbstractService { + + /** + * Create a new instance of AHSClient. + */ + @Public + public static AHSClient createAHSClient() { + AHSClient client = new AHSClientImpl(); + return client; + } + + @Private + public AHSClient(String name) { + super(name); + } + + /** + *

+ * Get a report of the given Application. + *

+ * + *

+ * In secure mode, YARN verifies access to the application, queue + * etc. before accepting the request. + *

+ * + *

+ * If the user does not have VIEW_APP access then the following + * fields in the report will be set to stubbed values: + *

    + *
  • host - set to "N/A"
  • + *
  • RPC port - set to -1
  • + *
  • client token - set to "N/A"
  • + *
  • diagnostics - set to "N/A"
  • + *
  • tracking URL - set to "N/A"
  • + *
  • original tracking URL - set to "N/A"
  • + *
  • resource usage report - all values are -1
  • + *
+ *

+ * + * @param appId + * {@link ApplicationId} of the application that needs a report + * @return application report + * @throws YarnException + * @throws IOException + */ + public abstract ApplicationReport getApplicationReport(ApplicationId appId) + throws YarnException, IOException; + + /** + *

+ * Get a report (ApplicationReport) of all Applications in the cluster. + *

+ * + *

+ * If the user does not have VIEW_APP access for an application + * then the corresponding report will be filtered as described in + * {@link #getApplicationReport(ApplicationId)}. + *

+ * + * @return a list of reports for all applications + * @throws YarnException + * @throws IOException + */ + public abstract List getApplications() + throws YarnException, IOException; + + /** + *

+ * Get a report of the given ApplicationAttempt. + *

+ * + *

+ * In secure mode, YARN verifies access to the application, queue + * etc. before accepting the request. + *

+ * + * @param applicationAttemptId + * {@link ApplicationAttemptId} of the application attempt that needs + * a report + * @return application attempt report + * @throws YarnException + * @throws {@link ApplicationAttemptNotFoundException} if application attempt + * not found + * @throws IOException + */ + public abstract ApplicationAttemptReport getApplicationAttemptReport( + ApplicationAttemptId appAttemptId) throws YarnException, IOException; + + /** + *

+ * Get a report of all (ApplicationAttempts) of Application in the cluster. + *

+ * + * @param applicationId + * @return a list of reports for all application attempts for specified + * application + * @throws YarnException + * @throws IOException + */ + public abstract List getApplicationAttempts( + ApplicationId appId) throws YarnException, IOException; + + /** + *

+ * Get a report of the given Container. + *

+ * + *

+ * In secure mode, YARN verifies access to the application, queue + * etc. before accepting the request. + *

+ * + * @param containerId + * {@link ContainerId} of the container that needs a report + * @return container report + * @throws YarnException + * @throws {@link ContainerNotFoundException} if container not found + * @throws IOException + */ + public abstract ContainerReport getContainerReport(ContainerId containerId) + throws YarnException, IOException; + + /** + *

+ * Get a report of all (Containers) of ApplicationAttempt in the cluster. + *

+ * + * @param applicationAttemptId + * @return a list of reports of all containers for specified application + * attempt + * @throws YarnException + * @throws IOException + */ + public abstract List getContainers( + ApplicationAttemptId applicationAttemptId) throws YarnException, + IOException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java index 155ba5d..d173cdf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/YarnClient.java @@ -29,9 +29,13 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.io.Text; import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.QueueInfo; @@ -40,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; +import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; @@ -360,4 +365,75 @@ public abstract QueueInfo getQueueInfo(String queueName) throws YarnException, */ public abstract List getQueueAclsInfo() throws YarnException, IOException; + + /** + *

+ * Get a report of the given ApplicationAttempt. + *

+ * + *

+ * In secure mode, YARN verifies access to the application, queue + * etc. before accepting the request. + *

+ * + * @param applicationAttemptId + * {@link ApplicationAttemptId} of the application attempt that needs + * a report + * @return application attempt report + * @throws YarnException + * @throws {@link ApplicationAttemptNotFoundException} if application attempt + * not found + * @throws IOException + */ + public abstract ApplicationAttemptReport getApplicationAttemptReport( + ApplicationAttemptId appAttemptId) throws YarnException, IOException; + + /** + *

+ * Get a report of all (ApplicationAttempts) of Application in the cluster. + *

+ * + * @param applicationId + * @return a list of reports for all application attempts for specified + * application. + * @throws YarnException + * @throws IOException + */ + public abstract List getApplicationAttempts( + ApplicationId appId) throws YarnException, IOException; + + /** + *

+ * Get a report of the given Container. + *

+ * + *

+ * In secure mode, YARN verifies access to the application, queue + * etc. before accepting the request. + *

+ * + * @param containerId + * {@link ContainerId} of the container that needs a report + * @return container report + * @throws YarnException + * @throws {@link ContainerNotFoundException} if container not found. + * @throws IOException + */ + public abstract ContainerReport getContainerReport(ContainerId containerId) + throws YarnException, IOException; + + /** + *

+ * Get a report of all (Containers) of ApplicationAttempt in the cluster. + *

+ * + * @param applicationAttemptId + * @return a list of reports of all containers for specified application + * attempts + * @throws YarnException + * @throws IOException + */ + public abstract List getContainers( + ApplicationAttemptId applicationAttemptId) throws YarnException, + IOException; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSClientImpl.java new file mode 100644 index 0000000..507cc11 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSClientImpl.java @@ -0,0 +1,156 @@ +/** + * 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.hadoop.yarn.client.api.impl; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.client.AHSProxy; +import org.apache.hadoop.yarn.client.api.AHSClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.util.Records; + +@Private +@Unstable +public class AHSClientImpl extends AHSClient { + + protected ApplicationHistoryProtocol ahsClient; + protected InetSocketAddress ahsAddress; + + public AHSClientImpl() { + super(AHSClientImpl.class.getName()); + } + + private static InetSocketAddress getAHSAddress(Configuration conf) { + return conf.getSocketAddr(YarnConfiguration.AHS_ADDRESS, + YarnConfiguration.DEFAULT_AHS_ADDRESS, + YarnConfiguration.DEFAULT_AHS_PORT); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + this.ahsAddress = getAHSAddress(conf); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + try { + ahsClient = AHSProxy.createAHSProxy(getConfig(), + ApplicationHistoryProtocol.class, this.ahsAddress); + } catch (IOException e) { + throw new YarnRuntimeException(e); + } + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + if (this.ahsClient != null) { + RPC.stopProxy(this.ahsClient); + } + super.serviceStop(); + } + + @Override + public ApplicationReport getApplicationReport(ApplicationId appId) + throws YarnException, IOException { + GetApplicationReportRequest request = GetApplicationReportRequest + .newInstance(appId); + GetApplicationReportResponse response = ahsClient + .getApplicationReport(request); + return response.getApplicationReport(); + } + + @Override + public List getApplications() throws YarnException, + IOException { + GetApplicationsRequest request = GetApplicationsRequest.newInstance(null, + null); + GetApplicationsResponse response = ahsClient.getApplications(request); + return response.getApplicationList(); + } + + @Override + public ApplicationAttemptReport getApplicationAttemptReport( + ApplicationAttemptId applicationAttemptId) throws YarnException, + IOException { + GetApplicationAttemptReportRequest request = GetApplicationAttemptReportRequest + .newInstance(applicationAttemptId); + GetApplicationAttemptReportResponse response = ahsClient + .getApplicationAttemptReport(request); + return response.getApplicationAttemptReport(); + } + + @Override + public List getApplicationAttempts( + ApplicationId appId) throws YarnException, IOException { + GetApplicationAttemptsRequest request = GetApplicationAttemptsRequest + .newInstance(appId); + GetApplicationAttemptsResponse response = ahsClient + .getApplicationAttempts(request); + return response.getApplicationAttemptList(); + } + + @Override + public ContainerReport getContainerReport(ContainerId containerId) + throws YarnException, IOException { + GetContainerReportRequest request = GetContainerReportRequest + .newInstance(containerId); + GetContainerReportResponse response = ahsClient.getContainerReport(request); + return response.getContainerReport(); + } + + @Override + public List getContainers( + ApplicationAttemptId applicationAttemptId) throws YarnException, + IOException { + GetContainersRequest request = GetContainersRequest + .newInstance(applicationAttemptId); + GetContainersResponse response = ahsClient.getContainers(request); + return response.getContainerList(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java index a5ff9f6..23a62de 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java @@ -49,9 +49,13 @@ import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.QueueInfo; @@ -60,9 +64,11 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; import org.apache.hadoop.yarn.client.ClientRMProxy; +import org.apache.hadoop.yarn.client.api.AHSClient; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; @@ -80,6 +86,7 @@ protected ApplicationClientProtocol rmClient; protected long submitPollIntervalMillis; private long asyncApiPollIntervalMillis; + protected AHSClient historyClient; private static final String ROOT = "root"; @@ -100,6 +107,8 @@ protected void serviceInit(Configuration conf) throws Exception { YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS, YarnConfiguration.DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS); } + historyClient = AHSClientImpl.createAHSClient(); + historyClient.init(getConfig()); super.serviceInit(conf); } @@ -107,7 +116,8 @@ protected void serviceInit(Configuration conf) throws Exception { protected void serviceStart() throws Exception { try { rmClient = ClientRMProxy.createRMProxy(getConfig(), - ApplicationClientProtocol.class); + ApplicationClientProtocol.class); + historyClient.start(); } catch (IOException e) { throw new YarnRuntimeException(e); } @@ -119,6 +129,7 @@ protected void serviceStop() throws Exception { if (this.rmClient != null) { RPC.stopProxy(this.rmClient); } + historyClient.stop(); super.serviceStop(); } @@ -207,11 +218,20 @@ public void killApplication(ApplicationId applicationId) @Override public ApplicationReport getApplicationReport(ApplicationId appId) throws YarnException, IOException { - GetApplicationReportRequest request = - Records.newRecord(GetApplicationReportRequest.class); - request.setApplicationId(appId); - GetApplicationReportResponse response = - rmClient.getApplicationReport(request); + GetApplicationReportResponse response = null; + try { + GetApplicationReportRequest request = Records + .newRecord(GetApplicationReportRequest.class); + request.setApplicationId(appId); + response = rmClient.getApplicationReport(request); + } catch (YarnException e) { + if (!(e.getClass() == ApplicationNotFoundException.class)) { + throw e; + } + } + if (response == null || response.getApplicationReport() == null) { + return historyClient.getApplicationReport(appId); + } return response.getApplicationReport(); } @@ -373,4 +393,29 @@ private void getChildQueues(QueueInfo parent, List queues, public void setRMClient(ApplicationClientProtocol rmClient) { this.rmClient = rmClient; } + + @Override + public ApplicationAttemptReport getApplicationAttemptReport( + ApplicationAttemptId appAttemptId) throws YarnException, IOException { + return historyClient.getApplicationAttemptReport(appAttemptId); + } + + @Override + public List getApplicationAttempts( + ApplicationId appId) throws YarnException, IOException { + return historyClient.getApplicationAttempts(appId); + } + + @Override + public ContainerReport getContainerReport(ContainerId containerId) + throws YarnException, IOException { + return historyClient.getContainerReport(containerId); + } + + @Override + public List getContainers( + ApplicationAttemptId applicationAttemptId) throws YarnException, + IOException { + return historyClient.getContainers(applicationAttemptId); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java index a7b7d65..9b465b7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java @@ -35,8 +35,10 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.util.ToolRunner; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.ConverterUtils; @@ -46,13 +48,22 @@ @Private @Unstable public class ApplicationCLI extends YarnCLI { - private static final String APPLICATIONS_PATTERN = - "%30s\t%20s\t%20s\t%10s\t%10s\t%18s\t%18s\t%15s\t%35s" + - System.getProperty("line.separator"); + private static final String APPLICATIONS_PATTERN = + "%30s\t%20s\t%20s\t%10s\t%10s\t%18s\t%18s\t%15s\t%35s" + + System.getProperty("line.separator"); + private static final String APPLICATION_ATTEMPTS_PATTERN = + "%30s\t%20s\t%35s\t%35s" + + System.getProperty("line.separator"); + private static final String CONTAINER_PATTERN = + "%30s\t%20s\t%20s\t%20s\t%20s\t%35s" + + System.getProperty("line.separator"); private static final String APP_TYPE_CMD = "appTypes"; - private static final String APP_STATE_CMD ="appStates"; + private static final String APP_STATE_CMD = "appStates"; private static final String ALLSTATES_OPTION = "ALL"; + public static final String APPLICATION = "application"; + public static final String APPLICATION_ATTEMPT = "applicationattempt"; + public static final String CONTAINER = "container"; private boolean allAppStates; @@ -69,23 +80,33 @@ public static void main(String[] args) throws Exception { public int run(String[] args) throws Exception { Options opts = new Options(); - opts.addOption(STATUS_CMD, true, "Prints the status of the application."); - opts.addOption(LIST_CMD, false, "List applications from the RM. " + - "Supports optional use of -appTypes to filter applications " + - "based on application type, " + - "and -appStates to filter applications based on application state"); + opts.addOption(STATUS_CMD, true, + "Prints the status of the application."); + if (args.length > 0 + && args[0].compareToIgnoreCase(APPLICATION_ATTEMPT) == 0) { + opts.addOption(LIST_CMD, true, + "List application attempts for aplication from AHS. "); + } else if (args.length > 0 && args[0].compareToIgnoreCase("container") == 0) { + opts.addOption(LIST_CMD, true, + "List containers for application attempts from AHS. "); + } else { + opts.addOption(LIST_CMD, false, "List applications from the RM. " + + "Supports optional use of -appTypes to filter applications " + + "based on application type, " + + "and -appStates to filter applications based on application state"); + } opts.addOption(KILL_CMD, true, "Kills the application."); opts.addOption(HELP_CMD, false, "Displays help for all commands."); - Option appTypeOpt = new Option(APP_TYPE_CMD, true, "Works with -list to " + - "filter applications based on " + - "input comma-separated list of application types."); + Option appTypeOpt = new Option(APP_TYPE_CMD, true, "Works with -list to " + + "filter applications based on " + + "input comma-separated list of application types."); appTypeOpt.setValueSeparator(','); appTypeOpt.setArgs(Option.UNLIMITED_VALUES); appTypeOpt.setArgName("Types"); opts.addOption(appTypeOpt); - Option appStateOpt = new Option(APP_STATE_CMD, true, "Works with -list " + - "to filter applications based on input comma-separated list of " + - "application states. " + getAllValidApplicationStates()); + Option appStateOpt = new Option(APP_STATE_CMD, true, "Works with -list " + + "to filter applications based on input comma-separated list of " + + "application states. " + getAllValidApplicationStates()); appStateOpt.setValueSeparator(','); appStateOpt.setArgs(Option.UNLIMITED_VALUES); appStateOpt.setArgName("States"); @@ -104,50 +125,77 @@ public int run(String[] args) throws Exception { } if (cliParser.hasOption(STATUS_CMD)) { - if (args.length != 2) { + if ((args[0].compareToIgnoreCase(APPLICATION) == 0) + || (args[0].compareToIgnoreCase(APPLICATION_ATTEMPT) == 0) + || (args[0].compareToIgnoreCase(CONTAINER) == 0)) { + if (args.length != 3) { + printUsage(opts); + return exitCode; + } + } else if (args.length != 2) { printUsage(opts); return exitCode; } - printApplicationReport(cliParser.getOptionValue(STATUS_CMD)); + if (args[0].compareToIgnoreCase(APPLICATION_ATTEMPT) == 0) { + printApplicationAttemptReport(cliParser.getOptionValue(STATUS_CMD)); + } else if (args[0].compareToIgnoreCase(CONTAINER) == 0) { + printContainerReport(cliParser.getOptionValue(STATUS_CMD)); + } else { + printApplicationReport(cliParser.getOptionValue(STATUS_CMD)); + } } else if (cliParser.hasOption(LIST_CMD)) { - allAppStates = false; - Set appTypes = new HashSet(); - if(cliParser.hasOption(APP_TYPE_CMD)) { - String[] types = cliParser.getOptionValues(APP_TYPE_CMD); - if (types != null) { - for (String type : types) { - if (!type.trim().isEmpty()) { - appTypes.add(type.toUpperCase().trim()); + if (args[0].compareToIgnoreCase(APPLICATION_ATTEMPT) == 0) { + if (args.length != 3) { + printUsage(opts); + return exitCode; + } + listApplicationAttempts(cliParser.getOptionValue(LIST_CMD)); + } else if (args[0].compareToIgnoreCase(CONTAINER) == 0) { + if (args.length != 3) { + printUsage(opts); + return exitCode; + } + listContainers(cliParser.getOptionValue(LIST_CMD)); + } else { + allAppStates = false; + Set appTypes = new HashSet(); + if (cliParser.hasOption(APP_TYPE_CMD)) { + String[] types = cliParser.getOptionValues(APP_TYPE_CMD); + if (types != null) { + for (String type : types) { + if (!type.trim().isEmpty()) { + appTypes.add(type.toUpperCase().trim()); + } } } } - } - EnumSet appStates = - EnumSet.noneOf(YarnApplicationState.class); - if (cliParser.hasOption(APP_STATE_CMD)) { - String[] states = cliParser.getOptionValues(APP_STATE_CMD); - if (states != null) { - for (String state : states) { - if (!state.trim().isEmpty()) { - if (state.trim().equalsIgnoreCase(ALLSTATES_OPTION)) { - allAppStates = true; - break; - } - try { - appStates.add(YarnApplicationState.valueOf(state.toUpperCase() - .trim())); - } catch (IllegalArgumentException ex) { - sysout.println("The application state " + state - + " is invalid."); - sysout.println(getAllValidApplicationStates()); - return exitCode; + EnumSet appStates = EnumSet + .noneOf(YarnApplicationState.class); + if (cliParser.hasOption(APP_STATE_CMD)) { + String[] states = cliParser.getOptionValues(APP_STATE_CMD); + if (states != null) { + for (String state : states) { + if (!state.trim().isEmpty()) { + if (state.trim().equalsIgnoreCase(ALLSTATES_OPTION)) { + allAppStates = true; + break; + } + try { + appStates.add(YarnApplicationState.valueOf(state + .toUpperCase().trim())); + } catch (IllegalArgumentException ex) { + sysout.println("The application state " + state + + " is invalid."); + sysout.println(getAllValidApplicationStates()); + return exitCode; + } } } } } + listApplications(appTypes, appStates); } - listApplications(appTypes, appStates); } else if (cliParser.hasOption(KILL_CMD)) { if (args.length != 2) { printUsage(opts); @@ -175,8 +223,85 @@ void printUsage(Options opts) { } /** - * Lists the applications matching the given application Types - * And application States present in the Resource Manager + * Prints the application attempt report for an application attempt id. + * + * @param applicationAttemptId + * @throws YarnException + */ + private void printApplicationAttemptReport(String applicationAttemptId) + throws YarnException, IOException { + ApplicationAttemptReport appAttemptReport = client + .getApplicationAttemptReport(ConverterUtils + .toApplicationAttemptId(applicationAttemptId)); + // Use PrintWriter.println, which uses correct platform line ending. + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintWriter appAttemptReportStr = new PrintWriter(baos); + if (appAttemptReport != null) { + appAttemptReportStr.println("Application Attempt Report : "); + appAttemptReportStr.print("\tApplicationAttempt-Id : "); + appAttemptReportStr.println(appAttemptReport.getApplicationAttemptId()); + appAttemptReportStr.print("\tState : "); + appAttemptReportStr.println(appAttemptReport + .getYarnApplicationAttemptState()); + appAttemptReportStr.print("\tAMContainer : "); + appAttemptReportStr.println(appAttemptReport.getAMContainerId() + .toString()); + appAttemptReportStr.print("\tTracking-URL : "); + appAttemptReportStr.println(appAttemptReport.getTrackingUrl()); + appAttemptReportStr.print("\tRPC Port : "); + appAttemptReportStr.println(appAttemptReport.getRpcPort()); + appAttemptReportStr.print("\tAM Host : "); + appAttemptReportStr.println(appAttemptReport.getHost()); + appAttemptReportStr.print("\tDiagnostics : "); + appAttemptReportStr.print(appAttemptReport.getDiagnostics()); + } else { + appAttemptReportStr.print("Application Attempt with id '" + + applicationAttemptId + "' doesn't exist in History Server."); + } + appAttemptReportStr.close(); + sysout.println(baos.toString("UTF-8")); + } + + /** + * Prints the container report for an container id. + * + * @param containerId + * @throws YarnException + */ + private void printContainerReport(String containerId) throws YarnException, + IOException { + ContainerReport containerReport = client.getContainerReport((ConverterUtils + .toContainerId(containerId))); + // Use PrintWriter.println, which uses correct platform line ending. + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintWriter containerReportStr = new PrintWriter(baos); + if (containerReport != null) { + containerReportStr.println("Container Report : "); + containerReportStr.print("\tContainer-Id : "); + containerReportStr.println(containerReport.getContainerId()); + containerReportStr.print("\tStart-Time : "); + containerReportStr.println(containerReport.getStartTime()); + containerReportStr.print("\tFinish-Time : "); + containerReportStr.println(containerReport.getFinishTime()); + containerReportStr.print("\tState : "); + containerReportStr.println(containerReport.getContainerState()); + containerReportStr.print("\tLOG-URL : "); + containerReportStr.println(containerReport.getLogUrl()); + containerReportStr.print("\tHost : "); + containerReportStr.println(containerReport.getAssignedNode()); + containerReportStr.print("\tDiagnostics : "); + containerReportStr.print(containerReport.getDiagnosticsInfo()); + } else { + containerReportStr.print("Container with id '" + containerId + + "' doesn't exist in Hostory Server."); + } + containerReportStr.close(); + sysout.println(baos.toString("UTF-8")); + } + + /** + * Lists the applications matching the given application Types And application + * States present in the Resource Manager * * @param appTypes * @param appStates @@ -188,7 +313,7 @@ private void listApplications(Set appTypes, IOException { PrintWriter writer = new PrintWriter(sysout); if (allAppStates) { - for(YarnApplicationState appState : YarnApplicationState.values()) { + for (YarnApplicationState appState : YarnApplicationState.values()) { appStates.add(appState); } } else { @@ -199,23 +324,24 @@ private void listApplications(Set appTypes, } } - List appsReport = - client.getApplications(appTypes, appStates); + List appsReport = client.getApplications(appTypes, + appStates); - writer - .println("Total number of applications (application-types: " + appTypes - + " and states: " + appStates + ")" + ":" + appsReport.size()); - writer.printf(APPLICATIONS_PATTERN, "Application-Id", - "Application-Name","Application-Type", "User", "Queue", - "State", "Final-State","Progress", "Tracking-URL"); + writer.println("Total number of applications (application-types: " + + appTypes + " and states: " + appStates + ")" + ":" + + appsReport.size()); + writer.printf(APPLICATIONS_PATTERN, "Application-Id", "Application-Name", + "Application-Type", "User", "Queue", "State", "Final-State", + "Progress", "Tracking-URL"); for (ApplicationReport appReport : appsReport) { DecimalFormat formatter = new DecimalFormat("###.##%"); String progress = formatter.format(appReport.getProgress()); writer.printf(APPLICATIONS_PATTERN, appReport.getApplicationId(), - appReport.getName(),appReport.getApplicationType(), appReport.getUser(), - appReport.getQueue(),appReport.getYarnApplicationState(), - appReport.getFinalApplicationStatus(),progress, - appReport.getOriginalTrackingUrl()); + appReport.getName(), appReport.getApplicationType(), appReport + .getUser(), appReport.getQueue(), appReport + .getYarnApplicationState(), + appReport.getFinalApplicationStatus(), progress, appReport + .getOriginalTrackingUrl()); } writer.flush(); } @@ -227,8 +353,8 @@ private void listApplications(Set appTypes, * @throws YarnException * @throws IOException */ - private void killApplication(String applicationId) - throws YarnException, IOException { + private void killApplication(String applicationId) throws YarnException, + IOException { ApplicationId appId = ConverterUtils.toApplicationId(applicationId); ApplicationReport appReport = client.getApplicationReport(appId); if (appReport.getYarnApplicationState() == YarnApplicationState.FINISHED @@ -296,14 +422,63 @@ private void printApplicationReport(String applicationId) private String getAllValidApplicationStates() { StringBuilder sb = new StringBuilder(); - sb.append("The valid application state can be" - + " one of the following: "); + sb.append("The valid application state can be" + " one of the following: "); sb.append(ALLSTATES_OPTION + ","); - for (YarnApplicationState appState : YarnApplicationState - .values()) { - sb.append(appState+","); + for (YarnApplicationState appState : YarnApplicationState.values()) { + sb.append(appState + ","); } String output = sb.toString(); - return output.substring(0, output.length()-1); + return output.substring(0, output.length() - 1); + } + + /** + * Lists the application attempts matching the given applicationid + * + * @param applicationId + * @throws YarnException + * @throws IOException + */ + private void listApplicationAttempts(String appId) throws YarnException, + IOException { + PrintWriter writer = new PrintWriter(sysout); + + List appAttemptsReport = client + .getApplicationAttempts(ConverterUtils.toApplicationId(appId)); + writer.println("Total number of application attempts " + ":" + + appAttemptsReport.size()); + writer.printf(APPLICATION_ATTEMPTS_PATTERN, "ApplicationAttempt-Id", + "State", "AM-Container-Id", "Tracking-URL"); + for (ApplicationAttemptReport appAttemptReport : appAttemptsReport) { + writer.printf(APPLICATION_ATTEMPTS_PATTERN, appAttemptReport + .getApplicationAttemptId(), appAttemptReport + .getYarnApplicationAttemptState(), appAttemptReport + .getAMContainerId().toString(), appAttemptReport.getTrackingUrl()); + } + writer.flush(); + } + + /** + * Lists the containers matching the given application attempts + * + * @param appAttemptId + * @throws YarnException + * @throws IOException + */ + private void listContainers(String appAttemptId) throws YarnException, + IOException { + PrintWriter writer = new PrintWriter(sysout); + + List appsReport = client + .getContainers(ConverterUtils.toApplicationAttemptId(appAttemptId)); + writer.println("Total number of containers " + ":" + appsReport.size()); + writer.printf(CONTAINER_PATTERN, "Container-Id", "Start Time", + "Finish Time", "State", "Host", "LOG-URL"); + for (ContainerReport containerReport : appsReport) { + writer.printf(CONTAINER_PATTERN, containerReport.getContainerId(), + containerReport.getStartTime(), containerReport.getFinishTime(), + containerReport.getContainerState(), containerReport + .getAssignedNode(), containerReport.getLogUrl()); + } + writer.flush(); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java new file mode 100644 index 0000000..2122b42 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java @@ -0,0 +1,394 @@ +/** + * 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.hadoop.yarn.client.api.impl; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.AHSClient; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.junit.Test; + +public class TestAHSClient { + + @Test + public void testClientStop() { + Configuration conf = new Configuration(); + AHSClient client = AHSClient.createAHSClient(); + client.init(conf); + client.start(); + client.stop(); + } + + @Test(timeout = 10000) + public void testGetApplications() throws YarnException, IOException { + Configuration conf = new Configuration(); + final AHSClient client = new MockAHSClient(); + client.init(conf); + client.start(); + + List expectedReports = ((MockAHSClient) client) + .getReports(); + + List reports = client.getApplications(); + Assert.assertEquals(reports, expectedReports); + + reports = client.getApplications(); + Assert.assertEquals(reports.size(), 4); + client.stop(); + } + + @Test(timeout = 10000) + public void testGetApplicationReport() throws YarnException, IOException { + Configuration conf = new Configuration(); + final AHSClient client = new MockAHSClient(); + client.init(conf); + client.start(); + + List expectedReports = ((MockAHSClient) client) + .getReports(); + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + ApplicationReport report = client.getApplicationReport(applicationId); + Assert.assertEquals(report, expectedReports.get(0)); + Assert.assertEquals(report.getApplicationId().toString(), expectedReports + .get(0).getApplicationId().toString()); + client.stop(); + } + + @Test(timeout = 10000) + public void testGetApplicationAttempts() throws YarnException, IOException { + Configuration conf = new Configuration(); + final AHSClient client = new MockAHSClient(); + client.init(conf); + client.start(); + + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + List reports = client + .getApplicationAttempts(applicationId); + Assert.assertNotNull(reports); + Assert.assertEquals(reports.get(0).getApplicationAttemptId(), + ApplicationAttemptId.newInstance(applicationId, 1)); + Assert.assertEquals(reports.get(1).getApplicationAttemptId(), + ApplicationAttemptId.newInstance(applicationId, 2)); + client.stop(); + } + + @Test(timeout = 10000) + public void testGetApplicationAttempt() throws YarnException, IOException { + Configuration conf = new Configuration(); + final AHSClient client = new MockAHSClient(); + client.init(conf); + client.start(); + + List expectedReports = ((MockAHSClient) client) + .getReports(); + + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance( + applicationId, 1); + ApplicationAttemptReport report = client + .getApplicationAttemptReport(appAttemptId); + Assert.assertNotNull(report); + Assert.assertEquals(report.getApplicationAttemptId().toString(), + expectedReports.get(0).getCurrentApplicationAttemptId().toString()); + client.stop(); + } + + @Test(timeout = 10000) + public void testGetContainers() throws YarnException, IOException { + Configuration conf = new Configuration(); + final AHSClient client = new MockAHSClient(); + client.init(conf); + client.start(); + + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance( + applicationId, 1); + List reports = client.getContainers(appAttemptId); + Assert.assertNotNull(reports); + Assert.assertEquals(reports.get(0).getContainerId(), (ContainerId + .newInstance(appAttemptId, 1))); + Assert.assertEquals(reports.get(1).getContainerId(), (ContainerId + .newInstance(appAttemptId, 2))); + client.stop(); + } + + @Test(timeout = 10000) + public void testGetContainerReport() throws YarnException, IOException { + Configuration conf = new Configuration(); + final AHSClient client = new MockAHSClient(); + client.init(conf); + client.start(); + + List expectedReports = ((MockAHSClient) client) + .getReports(); + + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance( + applicationId, 1); + ContainerId containerId = ContainerId.newInstance(appAttemptId, 1); + ContainerReport report = client.getContainerReport(containerId); + Assert.assertNotNull(report); + Assert.assertEquals(report.getContainerId().toString(), + (ContainerId.newInstance(expectedReports.get(0) + .getCurrentApplicationAttemptId(), 1)).toString()); + client.stop(); + } + + private static class MockAHSClient extends AHSClientImpl { + // private ApplicationReport mockReport; + private List reports = new ArrayList(); + private HashMap> attempts = + new HashMap>(); + private HashMap> containers = + new HashMap>(); + GetApplicationsResponse mockAppResponse = + mock(GetApplicationsResponse.class); + GetApplicationReportResponse mockResponse = + mock(GetApplicationReportResponse.class); + GetApplicationAttemptsResponse mockAppAttemptsResponse = + mock(GetApplicationAttemptsResponse.class); + GetApplicationAttemptReportResponse mockAttemptResponse = + mock(GetApplicationAttemptReportResponse.class); + GetContainersResponse mockContainersResponse = + mock(GetContainersResponse.class); + GetContainerReportResponse mockContainerResponse = + mock(GetContainerReportResponse.class); + + public MockAHSClient() { + super(); + createAppReports(); + } + + @Override + public void start() { + ahsClient = mock(ApplicationHistoryProtocol.class); + + try { + when( + ahsClient + .getApplicationReport(any(GetApplicationReportRequest.class))) + .thenReturn(mockResponse); + when(ahsClient.getApplications(any(GetApplicationsRequest.class))) + .thenReturn(mockAppResponse); + when( + ahsClient + .getApplicationAttemptReport(any(GetApplicationAttemptReportRequest.class))) + .thenReturn(mockAttemptResponse); + when( + ahsClient + .getApplicationAttempts(any(GetApplicationAttemptsRequest.class))) + .thenReturn(mockAppAttemptsResponse); + when(ahsClient.getContainers(any(GetContainersRequest.class))) + .thenReturn(mockContainersResponse); + + when(ahsClient.getContainerReport(any(GetContainerReportRequest.class))) + .thenReturn(mockContainerResponse); + + } catch (YarnException e) { + Assert.fail("Exception is not expected."); + } catch (IOException e) { + Assert.fail("Exception is not expected."); + } + } + + @Override + public List getApplications() throws YarnException, + IOException { + when(mockAppResponse.getApplicationList()).thenReturn(reports); + return super.getApplications(); + } + + @Override + public ApplicationReport getApplicationReport(ApplicationId appId) + throws YarnException, IOException { + when(mockResponse.getApplicationReport()).thenReturn(getReport(appId)); + return super.getApplicationReport(appId); + } + + @Override + public List getApplicationAttempts( + ApplicationId appId) throws YarnException, IOException { + when(mockAppAttemptsResponse.getApplicationAttemptList()).thenReturn( + getAttempts(appId)); + return super.getApplicationAttempts(appId); + } + + @Override + public ApplicationAttemptReport getApplicationAttemptReport( + ApplicationAttemptId appAttemptId) throws YarnException, IOException { + when(mockAttemptResponse.getApplicationAttemptReport()).thenReturn( + getAttempt(appAttemptId)); + return super.getApplicationAttemptReport(appAttemptId); + } + + @Override + public List getContainers(ApplicationAttemptId appAttemptId) + throws YarnException, IOException { + when(mockContainersResponse.getContainerList()).thenReturn( + getContainersReport(appAttemptId)); + return super.getContainers(appAttemptId); + } + + @Override + public ContainerReport getContainerReport(ContainerId containerId) + throws YarnException, IOException { + when(mockContainerResponse.getContainerReport()).thenReturn( + getContainer(containerId)); + return super.getContainerReport(containerId); + } + + @Override + public void stop() { + } + + public ApplicationReport getReport(ApplicationId appId) { + for (int i = 0; i < reports.size(); ++i) { + if (appId.toString().equalsIgnoreCase( + reports.get(i).getApplicationId().toString())) { + return reports.get(i); + } + } + return null; + } + + public List getAttempts(ApplicationId appId) { + return attempts.get(appId); + } + + public ApplicationAttemptReport getAttempt(ApplicationAttemptId appAttemptId) { + return attempts.get(appAttemptId.getApplicationId()).get(0); + } + + public List getContainersReport( + ApplicationAttemptId appAttemptId) { + return containers.get(appAttemptId); + } + + public ContainerReport getContainer(ContainerId containerId) { + return containers.get(containerId.getApplicationAttemptId()).get(0); + } + + public List getReports() { + return this.reports; + } + + private void createAppReports() { + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + ApplicationReport newApplicationReport = ApplicationReport + .newInstance(applicationId, ApplicationAttemptId.newInstance( + applicationId, 1), "user", "queue", "appname", "host", 124, null, + YarnApplicationState.RUNNING, "diagnostics", "url", 0, 0, + FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.53789f, "YARN", + null); + List applicationReports = new ArrayList(); + applicationReports.add(newApplicationReport); + List appAttempts = new ArrayList(); + ApplicationAttemptReport attempt = ApplicationAttemptReport.newInstance( + ApplicationAttemptId.newInstance(applicationId, 1), "host", 124, + "url", "diagnostics", YarnApplicationAttemptState.FINISHED, + ContainerId.newInstance(newApplicationReport + .getCurrentApplicationAttemptId(), 1)); + appAttempts.add(attempt); + ApplicationAttemptReport attempt1 = ApplicationAttemptReport.newInstance( + ApplicationAttemptId.newInstance(applicationId, 2), "host", 124, + "url", "diagnostics", YarnApplicationAttemptState.FINISHED, + ContainerId.newInstance(newApplicationReport + .getCurrentApplicationAttemptId(), 2)); + appAttempts.add(attempt1); + attempts.put(applicationId, appAttempts); + + List containerReports = new ArrayList(); + ContainerReport container = ContainerReport.newInstance(ContainerId + .newInstance(attempt.getApplicationAttemptId(), 1), null, NodeId + .newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678, + "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE); + containerReports.add(container); + + ContainerReport container1 = ContainerReport.newInstance(ContainerId + .newInstance(attempt.getApplicationAttemptId(), 2), null, NodeId + .newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678, + "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE); + containerReports.add(container1); + containers.put(attempt.getApplicationAttemptId(), containerReports); + + ApplicationId applicationId2 = ApplicationId.newInstance(1234, 6); + ApplicationReport newApplicationReport2 = ApplicationReport.newInstance( + applicationId2, ApplicationAttemptId.newInstance(applicationId2, 2), + "user2", "queue2", "appname2", "host2", 125, null, + YarnApplicationState.FINISHED, "diagnostics2", "url2", 2, 2, + FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.63789f, "NON-YARN", + null); + applicationReports.add(newApplicationReport2); + + ApplicationId applicationId3 = ApplicationId.newInstance(1234, 7); + ApplicationReport newApplicationReport3 = ApplicationReport.newInstance( + applicationId3, ApplicationAttemptId.newInstance(applicationId3, 3), + "user3", "queue3", "appname3", "host3", 126, null, + YarnApplicationState.RUNNING, "diagnostics3", "url3", 3, 3, + FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.73789f, "MAPREDUCE", + null); + applicationReports.add(newApplicationReport3); + + ApplicationId applicationId4 = ApplicationId.newInstance(1234, 8); + ApplicationReport newApplicationReport4 = ApplicationReport.newInstance( + applicationId4, ApplicationAttemptId.newInstance(applicationId4, 4), + "user4", "queue4", "appname4", "host4", 127, null, + YarnApplicationState.FAILED, "diagnostics4", "url4", 4, 4, + FinalApplicationStatus.SUCCEEDED, null, "N/A", 0.83789f, + "NON-MAPREDUCE", null); + applicationReports.add(newApplicationReport4); + reports = applicationReports; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java index 00ab789..dc6d98e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java @@ -58,12 +58,10 @@ import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.util.Records; import org.apache.log4j.Level; import org.apache.log4j.LogManager; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java index 1d08f24..dd6be0d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java @@ -43,19 +43,26 @@ import org.apache.commons.lang.time.DateFormatUtils; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.NodeState; +import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.apache.hadoop.yarn.util.Records; import org.junit.Before; import org.junit.Test; +import org.mortbay.log.Log; import org.apache.commons.cli.Options; @@ -114,19 +121,180 @@ public void testGetApplicationReport() throws Exception { } @Test + public void testGetApplicationAttemptReport() throws Exception { + ApplicationCLI cli = createAndGetAppCLI(); + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance( + applicationId, 1); + ApplicationAttemptReport attemptReport = ApplicationAttemptReport + .newInstance(attemptId, "host", 124, "url", "diagnostics", + YarnApplicationAttemptState.FINISHED, ContainerId.newInstance( + attemptId, 1)); + when( + client + .getApplicationAttemptReport(any(ApplicationAttemptId.class))) + .thenReturn(attemptReport); + int result = cli.run(new String[] { "applicationattempt", "-status", + attemptId.toString() }); + assertEquals(0, result); + verify(client).getApplicationAttemptReport(attemptId); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintWriter pw = new PrintWriter(baos); + pw.println("Application Attempt Report : "); + pw.println("\tApplicationAttempt-Id : appattempt_1234_0005_000001"); + pw.println("\tState : FINISHED"); + pw.println("\tAMContainer : container_1234_0005_01_000001"); + pw.println("\tTracking-URL : url"); + pw.println("\tRPC Port : 124"); + pw.println("\tAM Host : host"); + pw.println("\tDiagnostics : diagnostics"); + pw.close(); + String appReportStr = baos.toString("UTF-8"); + Assert.assertEquals(appReportStr, sysOutStream.toString()); + verify(sysOut, times(1)).println(isA(String.class)); + } + + @Test + public void testGetApplicationAttempts() throws Exception { + ApplicationCLI cli = createAndGetAppCLI(); + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance( + applicationId, 1); + ApplicationAttemptId attemptId1 = ApplicationAttemptId.newInstance( + applicationId, 2); + ApplicationAttemptReport attemptReport = ApplicationAttemptReport + .newInstance(attemptId, "host", 124, "url", "diagnostics", + YarnApplicationAttemptState.FINISHED, ContainerId.newInstance( + attemptId, 1)); + ApplicationAttemptReport attemptReport1 = ApplicationAttemptReport + .newInstance(attemptId1, "host", 124, "url", "diagnostics", + YarnApplicationAttemptState.FINISHED, ContainerId.newInstance( + attemptId1, 1)); + List reports = new ArrayList(); + reports.add(attemptReport); + reports.add(attemptReport1); + when(client.getApplicationAttempts(any(ApplicationId.class))) + .thenReturn(reports); + int result = cli.run(new String[] { "applicationattempt", "-list", + applicationId.toString() }); + assertEquals(0, result); + verify(client).getApplicationAttempts(applicationId); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintWriter pw = new PrintWriter(baos); + pw.println("Total number of application attempts :2"); + pw.print(" ApplicationAttempt-Id"); + pw.print("\t State"); + pw.print("\t AM-Container-Id"); + pw.println("\t Tracking-URL"); + pw.print(" appattempt_1234_0005_000001"); + pw.print("\t FINISHED"); + pw.print("\t container_1234_0005_01_000001"); + pw.println("\t url"); + pw.print(" appattempt_1234_0005_000002"); + pw.print("\t FINISHED"); + pw.print("\t container_1234_0005_02_000001"); + pw.println("\t url"); + pw.close(); + String appReportStr = baos.toString("UTF-8"); + Assert.assertEquals(appReportStr, sysOutStream.toString()); + } + + @Test + public void testGetContainerReport() throws Exception { + ApplicationCLI cli = createAndGetAppCLI(); + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance( + applicationId, 1); + ContainerId containerId = ContainerId.newInstance(attemptId, 1); + ContainerReport container = ContainerReport.newInstance(containerId, null, + NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678, + "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE); + when(client.getContainerReport(any(ContainerId.class))).thenReturn( + container); + int result = cli.run(new String[] { "container", "-status", + containerId.toString() }); + assertEquals(0, result); + verify(client).getContainerReport(containerId); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintWriter pw = new PrintWriter(baos); + pw.println("Container Report : "); + pw.println("\tContainer-Id : container_1234_0005_01_000001"); + pw.println("\tStart-Time : 1234"); + pw.println("\tFinish-Time : 5678"); + pw.println("\tState : COMPLETE"); + pw.println("\tLOG-URL : logURL"); + pw.println("\tHost : host:1234"); + pw.println("\tDiagnostics : diagnosticInfo"); + pw.close(); + String appReportStr = baos.toString("UTF-8"); + Assert.assertEquals(appReportStr, sysOutStream.toString()); + verify(sysOut, times(1)).println(isA(String.class)); + } + + @Test + public void testGetContainers() throws Exception { + ApplicationCLI cli = createAndGetAppCLI(); + ApplicationId applicationId = ApplicationId.newInstance(1234, 5); + ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance( + applicationId, 1); + ContainerId containerId = ContainerId.newInstance(attemptId, 1); + ContainerId containerId1 = ContainerId.newInstance(attemptId, 2); + ContainerReport container = ContainerReport.newInstance(containerId, null, + NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678, + "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE); + ContainerReport container1 = ContainerReport.newInstance(containerId1, null, + NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678, + "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE); + List reports = new ArrayList(); + reports.add(container); + reports.add(container1); + when(client.getContainers(any(ApplicationAttemptId.class))).thenReturn( + reports); + int result = cli.run(new String[] { "container", "-list", + attemptId.toString() }); + assertEquals(0, result); + verify(client).getContainers(attemptId); + Log.info(sysOutStream.toString()); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintWriter pw = new PrintWriter(baos); + pw.println("Total number of containers :2"); + pw.print(" Container-Id"); + pw.print("\t Start Time"); + pw.print("\t Finish Time"); + pw.print("\t State"); + pw.print("\t Host"); + pw.println("\t LOG-URL"); + pw.print(" container_1234_0005_01_000001"); + pw.print("\t 1234"); + pw.print("\t 5678"); + pw.print("\t COMPLETE"); + pw.print("\t host:1234"); + pw.println("\t logURL"); + pw.print(" container_1234_0005_01_000002"); + pw.print("\t 1234"); + pw.print("\t 5678"); + pw.print("\t COMPLETE"); + pw.print("\t host:1234"); + pw.println("\t logURL"); + pw.close(); + String appReportStr = baos.toString("UTF-8"); + Assert.assertEquals(appReportStr, sysOutStream.toString()); + } + + @Test public void testGetApplicationReportException() throws Exception { ApplicationCLI cli = createAndGetAppCLI(); ApplicationId applicationId = ApplicationId.newInstance(1234, 5); when(client.getApplicationReport(any(ApplicationId.class))).thenThrow( - new ApplicationNotFoundException("Application with id '" - + applicationId + "' doesn't exist in RM.")); + new ApplicationNotFoundException("History file for application" + + applicationId + " is not found")); try { cli.run(new String[] { "-status", applicationId.toString() }); Assert.fail(); } catch (Exception ex) { Assert.assertTrue(ex instanceof ApplicationNotFoundException); - Assert.assertEquals("Application with id '" + applicationId - + "' doesn't exist in RM.", ex.getMessage()); + Assert.assertEquals("History file for application" + + applicationId + " is not found", ex.getMessage()); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocolPB.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocolPB.java new file mode 100644 index 0000000..7765915 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocolPB.java @@ -0,0 +1,31 @@ +/** + * 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.hadoop.yarn.api; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.ipc.ProtocolInfo; +import org.apache.hadoop.yarn.proto.ApplicationHistoryProtocol.ApplicationHistoryProtocolService; + +@Private +@Unstable +@ProtocolInfo(protocolName = "org.apache.hadoop.yarn.api.ApplicationHistoryProtocolPB", protocolVersion = 1) +public interface ApplicationHistoryProtocolPB extends + ApplicationHistoryProtocolService.BlockingInterface { +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationHistoryProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationHistoryProtocolPBClientImpl.java new file mode 100644 index 0000000..d6c3181 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationHistoryProtocolPBClientImpl.java @@ -0,0 +1,231 @@ +/** + * 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.hadoop.yarn.api.impl.pb.client; + +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.ProtobufRpcEngine; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto; +import org.apache.hadoop.yarn.api.ApplicationClientProtocolPB; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocolPB; +import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptReportRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptReportResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptsRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptsResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerReportRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerReportResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainersRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainersResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetDelegationTokenRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetDelegationTokenResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.RPCUtil; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationsRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersRequestProto; + +import com.google.protobuf.ServiceException; + +public class ApplicationHistoryProtocolPBClientImpl implements + ApplicationHistoryProtocol, Closeable { + + private ApplicationHistoryProtocolPB proxy; + + public ApplicationHistoryProtocolPBClientImpl(long clientVersion, + InetSocketAddress addr, Configuration conf) throws IOException { + RPC.setProtocolEngine(conf, ApplicationHistoryProtocolPB.class, + ProtobufRpcEngine.class); + proxy = RPC.getProxy(ApplicationHistoryProtocolPB.class, clientVersion, + addr, conf); + } + + @Override + public void close() throws IOException { + if (this.proxy != null) { + RPC.stopProxy(this.proxy); + } + } + + @Override + public GetApplicationReportResponse getApplicationReport( + GetApplicationReportRequest request) throws YarnException, IOException { + GetApplicationReportRequestProto requestProto = ((GetApplicationReportRequestPBImpl) request) + .getProto(); + try { + return new GetApplicationReportResponsePBImpl(proxy.getApplicationReport( + null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public GetApplicationsResponse getApplications(GetApplicationsRequest request) + throws YarnException, IOException { + GetApplicationsRequestProto requestProto = ((GetApplicationsRequestPBImpl) request) + .getProto(); + try { + return new GetApplicationsResponsePBImpl(proxy.getApplications(null, + requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public GetApplicationAttemptReportResponse getApplicationAttemptReport( + GetApplicationAttemptReportRequest request) throws YarnException, + IOException { + GetApplicationAttemptReportRequestProto requestProto = ((GetApplicationAttemptReportRequestPBImpl) request) + .getProto(); + try { + return new GetApplicationAttemptReportResponsePBImpl(proxy + .getApplicationAttemptReport(null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public GetApplicationAttemptsResponse getApplicationAttempts( + GetApplicationAttemptsRequest request) throws YarnException, IOException { + GetApplicationAttemptsRequestProto requestProto = ((GetApplicationAttemptsRequestPBImpl) request) + .getProto(); + try { + return new GetApplicationAttemptsResponsePBImpl(proxy + .getApplicationAttempts(null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public GetContainerReportResponse getContainerReport( + GetContainerReportRequest request) throws YarnException, IOException { + GetContainerReportRequestProto requestProto = ((GetContainerReportRequestPBImpl) request) + .getProto(); + try { + return new GetContainerReportResponsePBImpl(proxy.getContainerReport( + null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public GetContainersResponse getContainers(GetContainersRequest request) + throws YarnException, IOException { + GetContainersRequestProto requestProto = ((GetContainersRequestPBImpl) request) + .getProto(); + try { + return new GetContainersResponsePBImpl(proxy.getContainers(null, + requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public GetDelegationTokenResponse getDelegationToken( + GetDelegationTokenRequest request) throws YarnException, + IOException { + GetDelegationTokenRequestProto requestProto = + ((GetDelegationTokenRequestPBImpl) request).getProto(); + try { + return new GetDelegationTokenResponsePBImpl(proxy.getDelegationToken( + null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public RenewDelegationTokenResponse renewDelegationToken( + RenewDelegationTokenRequest request) throws YarnException, + IOException { + RenewDelegationTokenRequestProto requestProto = + ((RenewDelegationTokenRequestPBImpl) request).getProto(); + try { + return new RenewDelegationTokenResponsePBImpl(proxy.renewDelegationToken( + null, requestProto)); + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } + + @Override + public CancelDelegationTokenResponse cancelDelegationToken( + CancelDelegationTokenRequest request) throws YarnException, + IOException { + CancelDelegationTokenRequestProto requestProto = + ((CancelDelegationTokenRequestPBImpl) request).getProto(); + try { + return new CancelDelegationTokenResponsePBImpl( + proxy.cancelDelegationToken(null, requestProto)); + + } catch (ServiceException e) { + RPCUtil.unwrapAndThrowException(e); + return null; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationHistoryProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationHistoryProtocolPBServiceImpl.java new file mode 100644 index 0000000..c375af2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationHistoryProtocolPBServiceImpl.java @@ -0,0 +1,234 @@ +/** + * 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.hadoop.yarn.api.impl.pb.service; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto; +import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto; +import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocolPB; +import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptReportRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptReportResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptsRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptsResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerReportRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerReportResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainersRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainersResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetDelegationTokenRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetDelegationTokenResponsePBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl; +import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationsRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationsResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersResponseProto; + +import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; + +@Private +public class ApplicationHistoryProtocolPBServiceImpl implements + ApplicationHistoryProtocolPB { + private ApplicationHistoryProtocol real; + + public ApplicationHistoryProtocolPBServiceImpl(ApplicationHistoryProtocol impl) { + this.real = impl; + } + + @Override + public GetApplicationReportResponseProto getApplicationReport( + RpcController arg0, GetApplicationReportRequestProto proto) + throws ServiceException { + GetApplicationReportRequestPBImpl request = new GetApplicationReportRequestPBImpl( + proto); + try { + GetApplicationReportResponse response = real + .getApplicationReport(request); + return ((GetApplicationReportResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public GetApplicationsResponseProto getApplications(RpcController controller, + GetApplicationsRequestProto proto) throws ServiceException { + GetApplicationsRequestPBImpl request = new GetApplicationsRequestPBImpl( + proto); + try { + GetApplicationsResponse response = real.getApplications(request); + return ((GetApplicationsResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public GetApplicationAttemptReportResponseProto getApplicationAttemptReport( + RpcController controller, GetApplicationAttemptReportRequestProto proto) + throws ServiceException { + GetApplicationAttemptReportRequestPBImpl request = new GetApplicationAttemptReportRequestPBImpl( + proto); + try { + GetApplicationAttemptReportResponse response = real + .getApplicationAttemptReport(request); + return ((GetApplicationAttemptReportResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public GetApplicationAttemptsResponseProto getApplicationAttempts( + RpcController controller, GetApplicationAttemptsRequestProto proto) + throws ServiceException { + GetApplicationAttemptsRequestPBImpl request = new GetApplicationAttemptsRequestPBImpl( + proto); + try { + GetApplicationAttemptsResponse response = real + .getApplicationAttempts(request); + return ((GetApplicationAttemptsResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public GetContainerReportResponseProto getContainerReport( + RpcController controller, GetContainerReportRequestProto proto) + throws ServiceException { + GetContainerReportRequestPBImpl request = new GetContainerReportRequestPBImpl( + proto); + try { + GetContainerReportResponse response = real.getContainerReport(request); + return ((GetContainerReportResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public GetContainersResponseProto getContainers(RpcController controller, + GetContainersRequestProto proto) throws ServiceException { + GetContainersRequestPBImpl request = new GetContainersRequestPBImpl(proto); + try { + GetContainersResponse response = real.getContainers(request); + return ((GetContainersResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public GetDelegationTokenResponseProto getDelegationToken( + RpcController controller, GetDelegationTokenRequestProto proto) + throws ServiceException { + GetDelegationTokenRequestPBImpl request = new GetDelegationTokenRequestPBImpl( + proto); + try { + GetDelegationTokenResponse response = real.getDelegationToken(request); + return ((GetDelegationTokenResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public RenewDelegationTokenResponseProto renewDelegationToken( + RpcController controller, RenewDelegationTokenRequestProto proto) + throws ServiceException { + RenewDelegationTokenRequestPBImpl request = new RenewDelegationTokenRequestPBImpl( + proto); + try { + RenewDelegationTokenResponse response = real + .renewDelegationToken(request); + return ((RenewDelegationTokenResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public CancelDelegationTokenResponseProto cancelDelegationToken( + RpcController controller, CancelDelegationTokenRequestProto proto) + throws ServiceException { + CancelDelegationTokenRequestPBImpl request = new CancelDelegationTokenRequestPBImpl( + proto); + try { + CancelDelegationTokenResponse response = real + .cancelDelegationToken(request); + return ((CancelDelegationTokenResponsePBImpl) response).getProto(); + } catch (YarnException e) { + throw new ServiceException(e); + } catch (IOException e) { + throw new ServiceException(e); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptReportRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptReportRequestPBImpl.java new file mode 100644 index 0000000..fe42419 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptReportRequestPBImpl.java @@ -0,0 +1,138 @@ +/** + * 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.hadoop.yarn.api.protocolrecords.impl.pb; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +@Private +@Unstable +public class GetApplicationAttemptReportRequestPBImpl extends + GetApplicationAttemptReportRequest { + + GetApplicationAttemptReportRequestProto proto = + GetApplicationAttemptReportRequestProto.getDefaultInstance(); + GetApplicationAttemptReportRequestProto.Builder builder = null; + boolean viaProto = false; + + private ApplicationAttemptId applicationAttemptId = null; + + public GetApplicationAttemptReportRequestPBImpl() { + builder = GetApplicationAttemptReportRequestProto.newBuilder(); + } + + public GetApplicationAttemptReportRequestPBImpl( + GetApplicationAttemptReportRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetApplicationAttemptReportRequestProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (applicationAttemptId != null) { + builder.setApplicationAttemptId( + convertToProtoFormat(this.applicationAttemptId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetApplicationAttemptReportRequestProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public ApplicationAttemptId getApplicationAttemptId() { + if (this.applicationAttemptId != null) { + return this.applicationAttemptId; + } + GetApplicationAttemptReportRequestProtoOrBuilder p + = viaProto ? proto : builder; + if (!p.hasApplicationAttemptId()) { + return null; + } + this.applicationAttemptId = + convertFromProtoFormat(p.getApplicationAttemptId()); + return this.applicationAttemptId; + } + + @Override + public void setApplicationAttemptId(ApplicationAttemptId applicationAttemptId) { + maybeInitBuilder(); + if (applicationAttemptId == null) { + builder.clearApplicationAttemptId(); + } + this.applicationAttemptId = applicationAttemptId; + } + + private ApplicationAttemptIdPBImpl convertFromProtoFormat( + ApplicationAttemptIdProto p) { + return new ApplicationAttemptIdPBImpl(p); + } + + private ApplicationAttemptIdProto convertToProtoFormat(ApplicationAttemptId t) { + return ((ApplicationAttemptIdPBImpl) t).getProto(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptReportResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptReportResponsePBImpl.java new file mode 100644 index 0000000..a9b9a96 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptReportResponsePBImpl.java @@ -0,0 +1,140 @@ +/** + * 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.hadoop.yarn.api.protocolrecords.impl.pb; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptReportPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportResponseProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +@Private +@Unstable +public class GetApplicationAttemptReportResponsePBImpl extends + GetApplicationAttemptReportResponse { + + GetApplicationAttemptReportResponseProto proto = + GetApplicationAttemptReportResponseProto.getDefaultInstance(); + GetApplicationAttemptReportResponseProto.Builder builder = null; + boolean viaProto = false; + + private ApplicationAttemptReport applicationAttemptReport = null; + + public GetApplicationAttemptReportResponsePBImpl() { + builder = GetApplicationAttemptReportResponseProto.newBuilder(); + } + + public GetApplicationAttemptReportResponsePBImpl( + GetApplicationAttemptReportResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetApplicationAttemptReportResponseProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.applicationAttemptReport != null) { + builder.setApplicationAttemptReport( + convertToProtoFormat(this.applicationAttemptReport)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetApplicationAttemptReportResponseProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public ApplicationAttemptReport getApplicationAttemptReport() { + if (this.applicationAttemptReport != null) { + return this.applicationAttemptReport; + } + GetApplicationAttemptReportResponseProtoOrBuilder p + = viaProto ? proto : builder; + if (!p.hasApplicationAttemptReport()) { + return null; + } + this.applicationAttemptReport = + convertFromProtoFormat(p.getApplicationAttemptReport()); + return this.applicationAttemptReport; + } + + @Override + public void setApplicationAttemptReport( + ApplicationAttemptReport ApplicationAttemptReport) { + maybeInitBuilder(); + if (ApplicationAttemptReport == null) { + builder.clearApplicationAttemptReport(); + } + this.applicationAttemptReport = ApplicationAttemptReport; + } + + private ApplicationAttemptReportPBImpl convertFromProtoFormat( + ApplicationAttemptReportProto p) { + return new ApplicationAttemptReportPBImpl(p); + } + + private ApplicationAttemptReportProto convertToProtoFormat( + ApplicationAttemptReport t) { + return ((ApplicationAttemptReportPBImpl) t).getProto(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptsRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptsRequestPBImpl.java new file mode 100644 index 0000000..4795d4b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptsRequestPBImpl.java @@ -0,0 +1,134 @@ +/** + * 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.hadoop.yarn.api.protocolrecords.impl.pb; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsRequestProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +@Private +@Unstable +public class GetApplicationAttemptsRequestPBImpl extends + GetApplicationAttemptsRequest { + + GetApplicationAttemptsRequestProto proto = + GetApplicationAttemptsRequestProto.getDefaultInstance(); + GetApplicationAttemptsRequestProto.Builder builder = null; + boolean viaProto = false; + + ApplicationId applicationId = null; + + public GetApplicationAttemptsRequestPBImpl() { + builder = GetApplicationAttemptsRequestProto.newBuilder(); + } + + public GetApplicationAttemptsRequestPBImpl( + GetApplicationAttemptsRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetApplicationAttemptsRequestProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (applicationId != null) { + builder.setApplicationId(convertToProtoFormat(this.applicationId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetApplicationAttemptsRequestProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public ApplicationId getApplicationId() { + if (this.applicationId != null) { + return this.applicationId; + } + GetApplicationAttemptsRequestProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationId()) { + return null; + } + this.applicationId = convertFromProtoFormat(p.getApplicationId()); + return this.applicationId; + } + + @Override + public void setApplicationId(ApplicationId applicationId) { + maybeInitBuilder(); + if (applicationId == null) { + builder.clearApplicationId(); + } + this.applicationId = applicationId; + } + + private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) { + return new ApplicationIdPBImpl(p); + } + + private ApplicationIdProto convertToProtoFormat(ApplicationId t) { + return ((ApplicationIdPBImpl) t).getProto(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptsResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptsResponsePBImpl.java new file mode 100644 index 0000000..67116d6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetApplicationAttemptsResponsePBImpl.java @@ -0,0 +1,186 @@ +/** + * 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.hadoop.yarn.api.protocolrecords.impl.pb; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptReportPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsResponseProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +@Private +@Unstable +public class GetApplicationAttemptsResponsePBImpl extends + GetApplicationAttemptsResponse { + + GetApplicationAttemptsResponseProto proto = + GetApplicationAttemptsResponseProto.getDefaultInstance(); + GetApplicationAttemptsResponseProto.Builder builder = null; + boolean viaProto = false; + + List applicationAttemptList; + + public GetApplicationAttemptsResponsePBImpl() { + builder = GetApplicationAttemptsResponseProto.newBuilder(); + } + + public GetApplicationAttemptsResponsePBImpl( + GetApplicationAttemptsResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + @Override + public List getApplicationAttemptList() { + initLocalApplicationAttemptsList(); + return this.applicationAttemptList; + } + + @Override + public void setApplicationAttemptList( + List applicationAttempts) { + maybeInitBuilder(); + if (applicationAttempts == null) { + builder.clearApplicationAttempts(); + } + this.applicationAttemptList = applicationAttempts; + } + + public GetApplicationAttemptsResponseProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.applicationAttemptList != null) { + addLocalApplicationAttemptsToProto(); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetApplicationAttemptsResponseProto.newBuilder(proto); + } + viaProto = false; + } + + // Once this is called. containerList will never be null - until a getProto + // is called. + private void initLocalApplicationAttemptsList() { + if (this.applicationAttemptList != null) { + return; + } + GetApplicationAttemptsResponseProtoOrBuilder p = viaProto ? proto : builder; + List list = p.getApplicationAttemptsList(); + applicationAttemptList = new ArrayList(); + + for (ApplicationAttemptReportProto a : list) { + applicationAttemptList.add(convertFromProtoFormat(a)); + } + } + + private void addLocalApplicationAttemptsToProto() { + maybeInitBuilder(); + builder.clearApplicationAttempts(); + if (applicationAttemptList == null) { + return; + } + Iterable iterable = + new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + + Iterator iter = + applicationAttemptList.iterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public ApplicationAttemptReportProto next() { + return convertToProtoFormat(iter.next()); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + + } + }; + + } + }; + builder.addAllApplicationAttempts(iterable); + } + + private ApplicationAttemptReportPBImpl convertFromProtoFormat( + ApplicationAttemptReportProto p) { + return new ApplicationAttemptReportPBImpl(p); + } + + private ApplicationAttemptReportProto convertToProtoFormat( + ApplicationAttemptReport t) { + return ((ApplicationAttemptReportPBImpl) t).getProto(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerReportRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerReportRequestPBImpl.java new file mode 100644 index 0000000..baa6721 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerReportRequestPBImpl.java @@ -0,0 +1,129 @@ +/** + * 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.hadoop.yarn.api.protocolrecords.impl.pb; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportRequestProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +@Private +@Unstable +public class GetContainerReportRequestPBImpl extends GetContainerReportRequest { + GetContainerReportRequestProto proto = + GetContainerReportRequestProto.getDefaultInstance(); + GetContainerReportRequestProto.Builder builder = null; + boolean viaProto = false; + + private ContainerId containerId = null; + + public GetContainerReportRequestPBImpl() { + builder = GetContainerReportRequestProto.newBuilder(); + } + + public GetContainerReportRequestPBImpl(GetContainerReportRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetContainerReportRequestProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (containerId != null) { + builder.setContainerId(convertToProtoFormat(this.containerId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) + maybeInitBuilder(); + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetContainerReportRequestProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public ContainerId getContainerId() { + if (this.containerId != null) { + return this.containerId; + } + GetContainerReportRequestProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasContainerId()) { + return null; + } + this.containerId = convertFromProtoFormat(p.getContainerId()); + return this.containerId; + } + + @Override + public void setContainerId(ContainerId containerId) { + maybeInitBuilder(); + if (containerId == null) { + builder.clearContainerId(); + } + this.containerId = containerId; + } + + private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) { + return new ContainerIdPBImpl(p); + } + + private ContainerIdProto convertToProtoFormat(ContainerId t) { + return ((ContainerIdPBImpl) t).getProto(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerReportResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerReportResponsePBImpl.java new file mode 100644 index 0000000..93fdbfc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerReportResponsePBImpl.java @@ -0,0 +1,129 @@ +/** + * 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.hadoop.yarn.api.protocolrecords.impl.pb; + +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerReportPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerReportProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportResponseProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +public class GetContainerReportResponsePBImpl + extends GetContainerReportResponse { + + GetContainerReportResponseProto proto = + GetContainerReportResponseProto.getDefaultInstance(); + GetContainerReportResponseProto.Builder builder = null; + boolean viaProto = false; + + private ContainerReport containerReport = null; + + public GetContainerReportResponsePBImpl() { + builder = GetContainerReportResponseProto.newBuilder(); + } + + public GetContainerReportResponsePBImpl( + GetContainerReportResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetContainerReportResponseProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.containerReport != null) { + builder.setContainerReport(convertToProtoFormat(this.containerReport)); + } + } + + private void mergeLocalToProto() { + if (viaProto) + maybeInitBuilder(); + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetContainerReportResponseProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public ContainerReport getContainerReport() { + if (this.containerReport != null) { + return this.containerReport; + } + GetContainerReportResponseProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasContainerReport()) { + return null; + } + this.containerReport = convertFromProtoFormat(p.getContainerReport()); + return this.containerReport; + } + + @Override + public void setContainerReport(ContainerReport containerReport) { + maybeInitBuilder(); + if (containerReport == null) { + builder.clearContainerReport(); + } + this.containerReport = containerReport; + } + + private ContainerReportPBImpl convertFromProtoFormat( + ContainerReportProto p) { + return new ContainerReportPBImpl(p); + } + + private ContainerReportProto convertToProtoFormat(ContainerReport t) { + return ((ContainerReportPBImpl) t).getProto(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainersRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainersRequestPBImpl.java new file mode 100644 index 0000000..5285977 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainersRequestPBImpl.java @@ -0,0 +1,131 @@ +/** + * 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.hadoop.yarn.api.protocolrecords.impl.pb; + +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersRequestProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersRequestProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +public class GetContainersRequestPBImpl extends GetContainersRequest { + GetContainersRequestProto proto = + GetContainersRequestProto.getDefaultInstance(); + GetContainersRequestProto.Builder builder = null; + boolean viaProto = false; + + private ApplicationAttemptId applicationAttemptId = null; + + public GetContainersRequestPBImpl() { + builder = GetContainersRequestProto.newBuilder(); + } + + public GetContainersRequestPBImpl(GetContainersRequestProto proto) { + this.proto = proto; + viaProto = true; + } + + public GetContainersRequestProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (applicationAttemptId != null) { + builder.setApplicationAttemptId( + convertToProtoFormat(this.applicationAttemptId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetContainersRequestProto.newBuilder(proto); + } + viaProto = false; + } + + @Override + public ApplicationAttemptId getApplicationAttemptId() { + if (this.applicationAttemptId != null) { + return this.applicationAttemptId; + } + GetContainersRequestProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationAttemptId()) { + return null; + } + this.applicationAttemptId = + convertFromProtoFormat(p.getApplicationAttemptId()); + return this.applicationAttemptId; + } + + @Override + public void setApplicationAttemptId( + ApplicationAttemptId applicationAttemptId) { + maybeInitBuilder(); + if (applicationAttemptId == null) { + builder.clearApplicationAttemptId(); + } + this.applicationAttemptId = applicationAttemptId; + } + + private ApplicationAttemptIdPBImpl convertFromProtoFormat( + ApplicationAttemptIdProto p) { + return new ApplicationAttemptIdPBImpl(p); + } + + private ApplicationAttemptIdProto convertToProtoFormat( + ApplicationAttemptId t) { + return ((ApplicationAttemptIdPBImpl) t).getProto(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainersResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainersResponsePBImpl.java new file mode 100644 index 0000000..080552b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainersResponsePBImpl.java @@ -0,0 +1,180 @@ +/** + * 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.hadoop.yarn.api.protocolrecords.impl.pb; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerReportPBImpl; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerReportProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersResponseProto; +import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersResponseProtoOrBuilder; + +import com.google.protobuf.TextFormat; + +@Private +@Unstable +public class GetContainersResponsePBImpl +extends GetContainersResponse { + + GetContainersResponseProto proto = + GetContainersResponseProto.getDefaultInstance(); + GetContainersResponseProto.Builder builder = null; + boolean viaProto = false; + + List containerList; + + public GetContainersResponsePBImpl() { + builder = GetContainersResponseProto.newBuilder(); + } + + public GetContainersResponsePBImpl(GetContainersResponseProto proto) { + this.proto = proto; + viaProto = true; + } + + @Override + public List getContainerList() { + initLocalContainerList(); + return this.containerList; + } + + @Override + public void setContainerList(List containers) { + maybeInitBuilder(); + if (containers == null) { + builder.clearContainers(); + } + this.containerList = containers; + } + + public GetContainersResponseProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.containerList != null) { + addLocalContainersToProto(); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = GetContainersResponseProto.newBuilder(proto); + } + viaProto = false; + } + + // Once this is called. containerList will never be null - until a getProto + // is called. + private void initLocalContainerList() { + if (this.containerList != null) { + return; + } + GetContainersResponseProtoOrBuilder p = viaProto ? proto : builder; + List list = p.getContainersList(); + containerList = new ArrayList(); + + for (ContainerReportProto c : list) { + containerList.add(convertFromProtoFormat(c)); + } + } + + private void addLocalContainersToProto() { + maybeInitBuilder(); + builder.clearContainers(); + if (containerList == null) { + return; + } + Iterable iterable = new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + + Iterator iter = containerList.iterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public ContainerReportProto next() { + return convertToProtoFormat(iter.next()); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + + } + }; + + } + }; + builder.addAllContainers(iterable); + } + + private ContainerReportPBImpl convertFromProtoFormat(ContainerReportProto p) { + return new ContainerReportPBImpl(p); + } + + private ContainerReportProto convertToProtoFormat(ContainerReport t) { + return ((ContainerReportPBImpl) t).getProto(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptReportPBImpl.java new file mode 100644 index 0000000..23a2096 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptReportPBImpl.java @@ -0,0 +1,270 @@ +/** + * 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.hadoop.yarn.api.records.impl.pb; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto; + +import com.google.protobuf.TextFormat; + +public class ApplicationAttemptReportPBImpl extends ApplicationAttemptReport { + ApplicationAttemptReportProto proto = + ApplicationAttemptReportProto.getDefaultInstance(); + ApplicationAttemptReportProto.Builder builder = null; + boolean viaProto = false; + + private ApplicationAttemptId ApplicationAttemptId; + private ContainerId amContainerId; + + public ApplicationAttemptReportPBImpl() { + builder = ApplicationAttemptReportProto.newBuilder(); + } + + public ApplicationAttemptReportPBImpl(ApplicationAttemptReportProto proto) { + this.proto = proto; + viaProto = true; + } + + @Override + public ApplicationAttemptId getApplicationAttemptId() { + if (this.ApplicationAttemptId != null) { + return this.ApplicationAttemptId; + } + + ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationAttemptId()) { + return null; + } + this.ApplicationAttemptId = + convertFromProtoFormat(p.getApplicationAttemptId()); + return this.ApplicationAttemptId; + } + + @Override + public String getHost() { + ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasHost()) { + return null; + } + return p.getHost(); + } + + @Override + public int getRpcPort() { + ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder; + return p.getRpcPort(); + } + + @Override + public String getTrackingUrl() { + ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasTrackingUrl()) { + return null; + } + return p.getTrackingUrl(); + } + + @Override + public String getDiagnostics() { + ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasDiagnostics()) { + return null; + } + return p.getDiagnostics(); + } + + @Override + public YarnApplicationAttemptState getYarnApplicationAttemptState() { + ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasYarnApplicationAttemptState()) { + return null; + } + return convertFromProtoFormat(p.getYarnApplicationAttemptState()); + } + + @Override + public void setYarnApplicationAttemptState(YarnApplicationAttemptState state) { + maybeInitBuilder(); + if (state == null) { + builder.clearYarnApplicationAttemptState(); + return; + } + builder.setYarnApplicationAttemptState(convertToProtoFormat(state)); + } + + private YarnApplicationAttemptStateProto convertToProtoFormat( + YarnApplicationAttemptState state) { + return ProtoUtils.convertToProtoFormat(state); + } + + private YarnApplicationAttemptState convertFromProtoFormat( + YarnApplicationAttemptStateProto yarnApplicationAttemptState) { + return ProtoUtils.convertFromProtoFormat(yarnApplicationAttemptState); + } + + @Override + public void setApplicationAttemptId(ApplicationAttemptId applicationAttemptId) { + maybeInitBuilder(); + if (applicationAttemptId == null) + builder.clearApplicationAttemptId(); + this.ApplicationAttemptId = applicationAttemptId; + } + + @Override + public void setHost(String host) { + maybeInitBuilder(); + if (host == null) { + builder.clearHost(); + return; + } + builder.setHost(host); + } + + @Override + public void setRpcPort(int rpcPort) { + maybeInitBuilder(); + builder.setRpcPort(rpcPort); + } + + @Override + public void setTrackingUrl(String url) { + maybeInitBuilder(); + if (url == null) { + builder.clearTrackingUrl(); + return; + } + builder.setTrackingUrl(url); + } + + @Override + public void setDiagnostics(String diagnostics) { + maybeInitBuilder(); + if (diagnostics == null) { + builder.clearDiagnostics(); + return; + } + builder.setDiagnostics(diagnostics); + } + + public ApplicationAttemptReportProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ApplicationAttemptReportProto.newBuilder(proto); + } + viaProto = false; + } + + private void mergeLocalToProto() { + if (viaProto) + maybeInitBuilder(); + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void mergeLocalToBuilder() { + if (this.ApplicationAttemptId != null && + !((ApplicationAttemptIdPBImpl) this.ApplicationAttemptId) + .getProto().equals(builder.getApplicationAttemptId())) { + builder.setApplicationAttemptId( + convertToProtoFormat(this.ApplicationAttemptId)); + } + + if (this.amContainerId != null + && !((ContainerIdPBImpl) this.amContainerId).getProto().equals( + builder.getAmContainerId())) { + builder.setAmContainerId(convertToProtoFormat(this.amContainerId)); + } + } + + private ContainerIdProto convertToProtoFormat(ContainerId amContainerId) { + return ((ContainerIdPBImpl)amContainerId).getProto(); + } + + private ContainerIdPBImpl convertFromProtoFormat( + ContainerIdProto amContainerId) { + return new ContainerIdPBImpl(amContainerId); + } + + private ApplicationAttemptIdProto + convertToProtoFormat(ApplicationAttemptId t) { + return ((ApplicationAttemptIdPBImpl) t).getProto(); + } + + private ApplicationAttemptIdPBImpl convertFromProtoFormat( + ApplicationAttemptIdProto applicationAttemptId) { + return new ApplicationAttemptIdPBImpl(applicationAttemptId); + } + + @Override + public ContainerId getAMContainerId() { + if (this.amContainerId != null) { + return this.amContainerId; + } + + ApplicationAttemptReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasAmContainerId()) { + return null; + } + this.amContainerId = + convertFromProtoFormat(p.getAmContainerId()); + return this.amContainerId; + } + + @Override + public void setAMContainerId(ContainerId amContainerId) { + maybeInitBuilder(); + if (amContainerId == null) + builder.clearAmContainerId(); + this.amContainerId = amContainerId; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java index ffaaf35..5b48141 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java @@ -298,19 +298,6 @@ public void setResource(Resource resource) { this.resource = resource; } - @Override - public void - setKeepContainersAcrossApplicationAttempts(boolean keepContainers) { - maybeInitBuilder(); - builder.setKeepContainersAcrossApplicationAttempts(keepContainers); - } - - @Override - public boolean getKeepContainersAcrossApplicationAttempts() { - ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder; - return p.getKeepContainersAcrossApplicationAttempts(); - } - private PriorityPBImpl convertFromProtoFormat(PriorityProto p) { return new PriorityPBImpl(p); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java new file mode 100644 index 0000000..f49839e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java @@ -0,0 +1,345 @@ +/** + * 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.hadoop.yarn.api.records.impl.pb; + +import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; + +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerReportProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerReportProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto; + +import com.google.protobuf.TextFormat; + +public class ContainerReportPBImpl extends ContainerReport { + + ContainerReportProto proto = ContainerReportProto.getDefaultInstance(); + ContainerReportProto.Builder builder = null; + boolean viaProto = false; + + private ContainerId containerId = null; + private Resource resource = null; + private NodeId nodeId = null; + private Priority priority = null; + + public ContainerReportPBImpl() { + builder = ContainerReportProto.newBuilder(); + } + + public ContainerReportPBImpl(ContainerReportProto proto) { + this.proto = proto; + viaProto = true; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + @Override + public Resource getAllocatedResource() { + if (this.resource != null) { + return this.resource; + } + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasResource()) { + return null; + } + this.resource = convertFromProtoFormat(p.getResource()); + return this.resource; + } + + @Override + public NodeId getAssignedNode() { + if (this.nodeId != null) { + return this.nodeId; + } + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasNodeId()) { + return null; + } + this.nodeId = convertFromProtoFormat(p.getNodeId()); + return this.nodeId; + } + + @Override + public ContainerId getContainerId() { + if (this.containerId != null) { + return this.containerId; + } + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasContainerId()) { + return null; + } + this.containerId = convertFromProtoFormat(p.getContainerId()); + return this.containerId; + } + + @Override + public String getDiagnosticsInfo() { + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasDiagnosticsInfo()) { + return null; + } + return (p.getDiagnosticsInfo()); + } + + @Override + public ContainerState getContainerState() { + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasContainerState()) { + return null; + } + return convertFromProtoFormat(p.getContainerState()); + } + + @Override + public long getFinishTime() { + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + return p.getFinishTime(); + } + + @Override + public String getLogUrl() { + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasLogUrl()) { + return null; + } + return (p.getLogUrl()); + } + + @Override + public Priority getPriority() { + if (this.priority != null) { + return this.priority; + } + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasPriority()) { + return null; + } + this.priority = convertFromProtoFormat(p.getPriority()); + return this.priority; + } + + @Override + public long getStartTime() { + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + return p.getStartTime(); + } + + @Override + public void setAllocatedResource(Resource resource) { + maybeInitBuilder(); + if (resource == null) + builder.clearResource(); + this.resource = resource; + } + + @Override + public void setAssignedNode(NodeId nodeId) { + maybeInitBuilder(); + if (nodeId == null) + builder.clearNodeId(); + this.nodeId = nodeId; + } + + @Override + public void setContainerId(ContainerId containerId) { + maybeInitBuilder(); + if (containerId == null) + builder.clearContainerId(); + this.containerId = containerId; + } + + @Override + public void setDiagnosticsInfo(String diagnosticsInfo) { + maybeInitBuilder(); + if (diagnosticsInfo == null) { + builder.clearDiagnosticsInfo(); + return; + } + builder.setDiagnosticsInfo(diagnosticsInfo); + } + + @Override + public void setContainerState(ContainerState containerState) { + maybeInitBuilder(); + if (containerState == null) { + builder.clearContainerState(); + return; + } + builder.setContainerState(convertToProtoFormat(containerState)); + } + + @Override + public int getContainerExitStatus() { + ContainerReportProtoOrBuilder p = viaProto ? proto : builder; + return p.getContainerExitStatus(); + } + + @Override + public void setContainerExitStatus(int containerExitStatus) { + maybeInitBuilder(); + builder.setContainerExitStatus(containerExitStatus); + } + + @Override + public void setFinishTime(long finishTime) { + maybeInitBuilder(); + builder.setFinishTime(finishTime); + } + + @Override + public void setLogUrl(String logUrl) { + maybeInitBuilder(); + if (logUrl == null) { + builder.clearLogUrl(); + return; + } + builder.setLogUrl(logUrl); + } + + @Override + public void setPriority(Priority priority) { + maybeInitBuilder(); + if (priority == null) { + builder.clearPriority(); + } + this.priority = priority; + } + + @Override + public void setStartTime(long startTime) { + maybeInitBuilder(); + builder.setStartTime(startTime); + } + + public ContainerReportProto getProto() { + + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return this.getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + private void mergeLocalToBuilder() { + if (this.containerId != null + && !((ContainerIdPBImpl) containerId).getProto().equals( + builder.getContainerId())) { + builder.setContainerId(convertToProtoFormat(this.containerId)); + } + if (this.nodeId != null + && !((NodeIdPBImpl) nodeId).getProto().equals(builder.getNodeId())) { + builder.setNodeId(convertToProtoFormat(this.nodeId)); + } + if (this.resource != null + && !((ResourcePBImpl) this.resource).getProto().equals( + builder.getResource())) { + builder.setResource(convertToProtoFormat(this.resource)); + } + if (this.priority != null + && !((PriorityPBImpl) this.priority).getProto().equals( + builder.getPriority())) { + builder.setPriority(convertToProtoFormat(this.priority)); + } + } + + private void mergeLocalToProto() { + if (viaProto) + maybeInitBuilder(); + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ContainerReportProto.newBuilder(proto); + } + viaProto = false; + } + + private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) { + return new ContainerIdPBImpl(p); + } + + private NodeIdPBImpl convertFromProtoFormat(NodeIdProto p) { + return new NodeIdPBImpl(p); + } + + private ContainerIdProto convertToProtoFormat(ContainerId t) { + return ((ContainerIdPBImpl) t).getProto(); + } + + private NodeIdProto convertToProtoFormat(NodeId t) { + return ((NodeIdPBImpl) t).getProto(); + } + + private ResourcePBImpl convertFromProtoFormat(ResourceProto p) { + return new ResourcePBImpl(p); + } + + private ResourceProto convertToProtoFormat(Resource t) { + return ((ResourcePBImpl) t).getProto(); + } + + private PriorityPBImpl convertFromProtoFormat(PriorityProto p) { + return new PriorityPBImpl(p); + } + + private PriorityProto convertToProtoFormat(Priority p) { + return ((PriorityPBImpl) p).getProto(); + } + + private ContainerStateProto convertToProtoFormat(ContainerState containerState) { + return ProtoUtils.convertToProtoFormat(containerState); + } + + private ContainerState convertFromProtoFormat( + ContainerStateProto containerState) { + return ProtoUtils.convertFromProtoFormat(containerState); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java index b660f7d..8d73744 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java @@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueState; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto; import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAccessTypeProto; @@ -45,6 +46,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.NodeStateProto; import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto; import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto; +import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto; import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto; import com.google.protobuf.ByteString; @@ -97,6 +99,21 @@ public static YarnApplicationState convertFromProtoFormat(YarnApplicationStatePr } /* + * YarnApplicationAttemptState + */ + private static String YARN_APPLICATION_ATTEMPT_STATE_PREFIX = "APP_ATTEMPT_"; + public static YarnApplicationAttemptStateProto convertToProtoFormat( + YarnApplicationAttemptState e) { + return YarnApplicationAttemptStateProto + .valueOf(YARN_APPLICATION_ATTEMPT_STATE_PREFIX + e.name()); + } + public static YarnApplicationAttemptState convertFromProtoFormat( + YarnApplicationAttemptStateProto e) { + return YarnApplicationAttemptState.valueOf(e.name().replace( + YARN_APPLICATION_ATTEMPT_STATE_PREFIX, "")); + } + + /* * ApplicationResourceUsageReport */ public static ApplicationResourceUsageReportProto convertToProtoFormat(ApplicationResourceUsageReport e) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AHSProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AHSProxy.java new file mode 100644 index 0000000..b254f07 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/AHSProxy.java @@ -0,0 +1,57 @@ +/** + * 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.hadoop.yarn.client; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.security.PrivilegedAction; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.ipc.YarnRPC; + +@InterfaceAudience.Public +@InterfaceStability.Evolving +@SuppressWarnings("unchecked") +public class AHSProxy { + + private static final Log LOG = LogFactory.getLog(AHSProxy.class); + + public static T createAHSProxy(final Configuration conf, + final Class protocol, InetSocketAddress ahsAddress) throws IOException { + LOG.info("Connecting to Application History server at " + ahsAddress); + return (T) getProxy(conf, protocol, ahsAddress); + } + + protected static T getProxy(final Configuration conf, + final Class protocol, final InetSocketAddress rmAddress) + throws IOException { + return UserGroupInformation.getCurrentUser().doAs( + new PrivilegedAction() { + @Override + public T run() { + return (T) YarnRPC.create(conf).getProxy(protocol, rmAddress, conf); + } + }); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java index fd285b4..a48b3c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/StringHelper.java @@ -178,4 +178,8 @@ private static void uappend(StringBuilder sb, String part) { public static String percent(double value) { return String.format("%.2f", value * 100); } + + public static String getPartUrl(String url, String part) { + return url.substring(url.indexOf(part)); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java index 1f59b87..91d2a20 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java @@ -24,6 +24,7 @@ public interface YarnWebParams { String NM_NODENAME = "nm.id"; String APPLICATION_ID = "app.id"; + String APPLICATION_ATTEMPT_ID = "appattempt.id"; String CONTAINER_ID = "container.id"; String CONTAINER_LOG_TYPE= "log.type"; String ENTITY_STRING = "entity.string"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java index 065d680..4a288c4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.yarn.webapp.util; +import static org.apache.hadoop.yarn.util.StringHelper.join; + import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; @@ -27,7 +29,9 @@ import org.apache.hadoop.http.HttpConfig; import org.apache.hadoop.http.HttpConfig.Policy; import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.util.ConverterUtils; import com.google.common.base.Joiner; @@ -144,6 +148,16 @@ public static String getNMWebAppURLWithoutScheme(Configuration conf) { YarnConfiguration.DEFAULT_NM_WEBAPP_ADDRESS); } } + + public static String getAHSWebAppURLWithoutScheme(Configuration conf) { + if (HttpConfig.isSecure()) { + return conf.get(YarnConfiguration.AHS_WEBAPP_HTTPS_ADDRESS, + YarnConfiguration.DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS); + } else { + return conf.get(YarnConfiguration.AHS_WEBAPP_ADDRESS, + YarnConfiguration.DEFAULT_AHS_WEBAPP_ADDRESS); + } + } /** * if url has scheme then it will be returned as it is else it will return @@ -160,4 +174,11 @@ public static String getURLWithScheme(String schemePrefix, String url) { return schemePrefix + url; } } + + public static String getLogUrl(String nodeHttpAddress, String allocatedNode, + ContainerId containerId, String user) { + return join(HttpConfig.getSchemePrefix(), nodeHttpAddress, "/logs", "/", + allocatedNode, "/", ConverterUtils.toString(containerId), "/", + ConverterUtils.toString(containerId), "/", user); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/applicationhistory/.keep b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/applicationhistory/.keep new file mode 100644 index 0000000..e69de29 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml index ead11eb..a138343 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml @@ -564,6 +564,27 @@ org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy + + Enable RM to write history data. If true, then + yarn.resourcemanager.history-writer.class must be specified + yarn.resourcemanager.history-writer.enabled + false + + + + Number of worker threads that write the history data. + yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size + 10 + + + + The implementation class of ApplicationHistoryStore, which is + to be used by RMApplicationHistoryWriter. + + yarn.resourcemanager.history-writer.class + org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore + + The hostname of the NM. @@ -1016,29 +1037,65 @@ - - CLASSPATH for YARN applications. A comma-separated list - of CLASSPATH entries. When this value is empty, the following default - CLASSPATH for YARN applications would be used. - For Linux: - $HADOOP_CONF_DIR, - $HADOOP_COMMON_HOME/share/hadoop/common/*, - $HADOOP_COMMON_HOME/share/hadoop/common/lib/*, - $HADOOP_HDFS_HOME/share/hadoop/hdfs/*, - $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*, - $HADOOP_YARN_HOME/share/hadoop/yarn/*, - $HADOOP_YARN_HOME/share/hadoop/yarn/lib/* - For Windows: - %HADOOP_CONF_DIR%, - %HADOOP_COMMON_HOME%/share/hadoop/common/*, - %HADOOP_COMMON_HOME%/share/hadoop/common/lib/*, - %HADOOP_HDFS_HOME%/share/hadoop/hdfs/*, - %HADOOP_HDFS_HOME%/share/hadoop/hdfs/lib/*, - %HADOOP_YARN_HOME%/share/hadoop/yarn/*, - %HADOOP_YARN_HOME%/share/hadoop/yarn/lib/* - - yarn.application.classpath - + CLASSPATH for YARN applications. A comma-separated list + of CLASSPATH entries + yarn.application.classpath + $HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/share/hadoop/common/*,$HADOOP_COMMON_HOME/share/hadoop/common/lib/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/*,$HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/*,$HADOOP_YARN_HOME/share/hadoop/yarn/*,$HADOOP_YARN_HOME/share/hadoop/yarn/lib/* + + + + + + The hostname of the AHS. + yarn.ahs.hostname + 0.0.0.0 + + + + The http address of the AHS web application. + yarn.ahs.webapp.address + ${yarn.ahs.hostname}:8188 + + + + The https adddress of the AHS web application. + yarn.ahs.webapp.https.address + ${yarn.ahs.hostname}:8190 + + + + URI pointing to the location of the FileSystem path where + the history will be persisted. This must be supplied when using + org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore + as the value for yarn.resourcemanager.history-writer.store.class + yarn.ahs.fs-history-store.uri + ${hadoop.log.dir}/yarn/system/ahstore + + + + This is default address for the Application History server + to start the RPC server. + yarn.ahs.address + 0.0.0.0:10200 + + + + CLient thread count to serve the client requests. + yarn.ahs.client.thread-count + 10 + + + + T-file compression types used to compress history data. + yarn.ahs.fs-history-store.compression-type + none + + + + Store class name for history store, defaulting to file + system store + yarn.ahs.store.class + org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml new file mode 100644 index 0000000..e4ab2aa --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml @@ -0,0 +1,44 @@ + + + + + hadoop-yarn-server + org.apache.hadoop + 3.0.0-SNAPSHOT + + 4.0.0 + org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + 3.0.0-SNAPSHOT + hadoop-yarn-server-applicationhistoryservice + + + + ${project.parent.parent.basedir} + + + + + org.apache.hadoop + hadoop-yarn-server-common + + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java new file mode 100644 index 0000000..0901d9d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java @@ -0,0 +1,205 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.Server; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse; +import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest; +import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException; +import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; +import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.ipc.YarnRPC; + +public class ApplicationHistoryClientService extends AbstractService { + private static final Log LOG = LogFactory + .getLog(ApplicationHistoryClientService.class); + private ApplicationHistoryManager history; + private ApplicationHistoryProtocol protocolHandler; + private Server server; + private InetSocketAddress bindAddress; + + public ApplicationHistoryClientService(ApplicationHistoryManager history) { + super("ApplicationHistoryClientService"); + this.history = history; + this.protocolHandler = new ApplicationHSClientProtocolHandler(); + } + + protected void serviceStart() throws Exception { + Configuration conf = getConfig(); + YarnRPC rpc = YarnRPC.create(conf); + InetSocketAddress address = conf.getSocketAddr( + YarnConfiguration.AHS_ADDRESS, YarnConfiguration.DEFAULT_AHS_ADDRESS, + YarnConfiguration.DEFAULT_AHS_PORT); + + server = rpc.getServer(ApplicationHistoryProtocol.class, protocolHandler, + address, conf, null, conf.getInt( + YarnConfiguration.AHS_CLIENT_THREAD_COUNT, + YarnConfiguration.DEFAULT_AHS_CLIENT_THREAD_COUNT)); + + server.start(); + this.bindAddress = conf.updateConnectAddr(YarnConfiguration.AHS_ADDRESS, + server.getListenerAddress()); + LOG.info("Instantiated ApplicationHistoryClientService at " + + this.bindAddress); + + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + if (server != null) { + server.stop(); + } + super.serviceStop(); + } + + @Private + public ApplicationHistoryProtocol getClientHandler() { + return this.protocolHandler; + } + + @Private + public InetSocketAddress getBindAddress() { + return this.bindAddress; + } + + private class ApplicationHSClientProtocolHandler implements + ApplicationHistoryProtocol { + + @Override + public CancelDelegationTokenResponse cancelDelegationToken( + CancelDelegationTokenRequest request) throws YarnException, IOException { + // TODO Auto-generated method stub + return null; + } + + @Override + public GetApplicationAttemptReportResponse getApplicationAttemptReport( + GetApplicationAttemptReportRequest request) throws YarnException, + IOException { + try { + GetApplicationAttemptReportResponse response = GetApplicationAttemptReportResponse + .newInstance(history.getApplicationAttempt(request + .getApplicationAttemptId())); + return response; + } catch (IOException e) { + throw new ApplicationAttemptNotFoundException(e.getMessage()); + } + } + + @Override + public GetApplicationAttemptsResponse getApplicationAttempts( + GetApplicationAttemptsRequest request) throws YarnException, + IOException { + GetApplicationAttemptsResponse response = GetApplicationAttemptsResponse + .newInstance(new ArrayList(history + .getApplicationAttempts(request.getApplicationId()).values())); + return response; + } + + @Override + public GetApplicationReportResponse getApplicationReport( + GetApplicationReportRequest request) throws YarnException, IOException { + try { + ApplicationId applicationId = request.getApplicationId(); + GetApplicationReportResponse response = GetApplicationReportResponse + .newInstance(history.getApplication(applicationId)); + return response; + } catch (IOException e) { + throw new ApplicationNotFoundException(e.getMessage()); + } + } + + @Override + public GetApplicationsResponse getApplications( + GetApplicationsRequest request) throws YarnException, IOException { + GetApplicationsResponse response = GetApplicationsResponse + .newInstance(new ArrayList(history + .getAllApplications().values())); + return response; + } + + @Override + public GetContainerReportResponse getContainerReport( + GetContainerReportRequest request) throws YarnException, IOException { + try { + GetContainerReportResponse response = GetContainerReportResponse + .newInstance(history.getContainer(request.getContainerId())); + return response; + } catch (IOException e) { + throw new ContainerNotFoundException(e.getMessage()); + } + } + + @Override + public GetContainersResponse getContainers(GetContainersRequest request) + throws YarnException, IOException { + GetContainersResponse response = GetContainersResponse + .newInstance(new ArrayList(history.getContainers( + request.getApplicationAttemptId()).values())); + return response; + } + + @Override + public GetDelegationTokenResponse getDelegationToken( + GetDelegationTokenRequest request) throws YarnException, IOException { + // TODO Auto-generated method stub + return null; + } + + @Override + public RenewDelegationTokenResponse renewDelegationToken( + RenewDelegationTokenRequest request) throws YarnException, IOException { + // TODO Auto-generated method stub + return null; + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java new file mode 100644 index 0000000..db25d29 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java @@ -0,0 +1,28 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.server.api.ApplicationContext; + +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface ApplicationHistoryManager extends ApplicationContext { +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java new file mode 100644 index 0000000..6ac8ed9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java @@ -0,0 +1,216 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerHistoryData; + +import com.google.common.annotations.VisibleForTesting; + +public class ApplicationHistoryManagerImpl extends AbstractService implements + ApplicationHistoryManager { + private static final Log LOG = LogFactory + .getLog(ApplicationHistoryManagerImpl.class); + private static final String UNAVAILABLE = "N/A"; + + private ApplicationHistoryStore historyStore; + + public ApplicationHistoryManagerImpl() { + super(ApplicationHistoryManagerImpl.class.getName()); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + LOG.info("ApplicationHistory Init"); + historyStore = createApplicationHistoryStore(conf); + historyStore.init(conf); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + LOG.info("Starting ApplicationHistory"); + historyStore.start(); + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + LOG.info("Stopping ApplicationHistory"); + historyStore.stop(); + super.serviceStop(); + } + + protected ApplicationHistoryStore createApplicationHistoryStore( + Configuration conf) { + return ReflectionUtils.newInstance(conf.getClass( + YarnConfiguration.AHS_STORE, FileSystemApplicationHistoryStore.class, + ApplicationHistoryStore.class), conf); + } + + @Override + public ContainerReport getAMContainer(ApplicationAttemptId appAttemptId) + throws IOException { + return convertToContainerReport(historyStore.getAMContainer(appAttemptId)); + } + + @Override + public Map getAllApplications() + throws IOException { + Map histData = historyStore + .getAllApplications(); + HashMap applicationsReport = new HashMap(); + for (ApplicationId appId : histData.keySet()) { + applicationsReport.put(appId, convertToApplicationReport(histData + .get(appId))); + } + return applicationsReport; + } + + @Override + public ApplicationReport getApplication(ApplicationId appId) + throws IOException { + return convertToApplicationReport(historyStore.getApplication(appId)); + } + + private ApplicationReport convertToApplicationReport( + ApplicationHistoryData appHistory) throws IOException { + ApplicationAttemptId currentApplicationAttemptId = null; + String trackingUrl = UNAVAILABLE; + String host = UNAVAILABLE; + int rpcPort = -1; + + ApplicationAttemptHistoryData lastAttempt = getLastAttempt(appHistory + .getApplicationId()); + if (lastAttempt != null) { + currentApplicationAttemptId = lastAttempt.getApplicationAttemptId(); + trackingUrl = lastAttempt.getTrackingURL(); + host = lastAttempt.getHost(); + rpcPort = lastAttempt.getRPCPort(); + } + return ApplicationReport.newInstance(appHistory.getApplicationId(), + currentApplicationAttemptId, appHistory.getUser(), appHistory + .getQueue(), appHistory.getApplicationName(), host, rpcPort, null, + appHistory.getYarnApplicationState(), appHistory.getDiagnosticsInfo(), + trackingUrl, appHistory.getStartTime(), appHistory.getFinishTime(), + appHistory.getFinalApplicationStatus(), null, "", 100, appHistory + .getApplicationType(), null); + } + + private ApplicationAttemptHistoryData getLastAttempt(ApplicationId appId) + throws IOException { + Map attempts = historyStore + .getApplicationAttempts(appId); + ApplicationAttemptId prevMaxAttemptId = null; + for (ApplicationAttemptId attemptId : attempts.keySet()) { + if (prevMaxAttemptId == null) { + prevMaxAttemptId = attemptId; + } else { + if (prevMaxAttemptId.getAttemptId() < attemptId.getAttemptId()) { + prevMaxAttemptId = attemptId; + } + } + } + return attempts.get(prevMaxAttemptId); + } + + private ApplicationAttemptReport convertToApplicationAttemptReport( + ApplicationAttemptHistoryData appAttemptHistory) { + return ApplicationAttemptReport.newInstance(appAttemptHistory + .getApplicationAttemptId(), appAttemptHistory.getHost(), + appAttemptHistory.getRPCPort(), appAttemptHistory.getTrackingURL(), + appAttemptHistory.getDiagnosticsInfo(), + appAttemptHistory.getYarnApplicationAttemptState(), + appAttemptHistory.getMasterContainerId()); + } + + @Override + public ApplicationAttemptReport getApplicationAttempt( + ApplicationAttemptId appAttemptId) throws IOException { + return convertToApplicationAttemptReport(historyStore + .getApplicationAttempt(appAttemptId)); + } + + @Override + public Map getApplicationAttempts( + ApplicationId appId) throws IOException { + Map histData = historyStore + .getApplicationAttempts(appId); + HashMap applicationAttemptsReport = new HashMap(); + for (ApplicationAttemptId appAttemptId : histData.keySet()) { + applicationAttemptsReport.put(appAttemptId, + convertToApplicationAttemptReport(histData.get(appAttemptId))); + } + return applicationAttemptsReport; + } + + @Override + public ContainerReport getContainer(ContainerId containerId) + throws IOException { + return convertToContainerReport(historyStore.getContainer(containerId)); + } + + private ContainerReport convertToContainerReport( + ContainerHistoryData containerHistory) { + return ContainerReport.newInstance(containerHistory.getContainerId(), + containerHistory.getAllocatedResource(), containerHistory + .getAssignedNode(), containerHistory.getPriority(), + containerHistory.getStartTime(), containerHistory.getFinishTime(), + containerHistory.getDiagnosticsInfo(), containerHistory.getLogURL(), + containerHistory.getContainerExitStatus(), containerHistory + .getContainerState()); + } + + @Override + public Map getContainers( + ApplicationAttemptId appAttemptId) throws IOException { + Map histData = historyStore + .getContainers(appAttemptId); + HashMap containersReport = new HashMap(); + for (ContainerId container : histData.keySet()) { + containersReport.put(container, convertToContainerReport(histData + .get(container))); + } + return containersReport; + } + + @Private + @VisibleForTesting + public ApplicationHistoryStore getHistoryStore() { + return this.historyStore; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryReader.java new file mode 100644 index 0000000..bd632d8 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryReader.java @@ -0,0 +1,109 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerHistoryData; + +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface ApplicationHistoryReader { + + /** + * This method returns Application {@link ApplicationHistoryData} for the + * specified {@link ApplicationId}. + * + * @return {@link ApplicationHistoryData} for the ApplicationId. + * @throws {@link IOException} + */ + ApplicationHistoryData getApplication(ApplicationId appId) throws IOException; + + /** + * This method returns all Application {@link ApplicationHistoryData}s + * + * @return map {@link ApplicationId, @link ApplicationHistoryData}s. + * @throws {@link IOException} + */ + Map getAllApplications() + throws IOException; + + /** + * Application can have multiple application attempts + * {@link ApplicationAttemptHistoryData}. This method returns the all + * {@link ApplicationAttemptHistoryData}s for the Application. + * + * @return all {@link ApplicationAttemptHistoryData}s for the Application. + * @throws {@link IOException} + */ + Map getApplicationAttempts( + ApplicationId appId) throws IOException; + + /** + * This method returns {@link ApplicationAttemptHistoryData} for specified + * {@link ApplicationId}. + * + * @param {@link ApplicationAttemptId} + * @return {@link ApplicationAttemptHistoryData} for ApplicationAttemptId + * @throws {@link IOException} + */ + ApplicationAttemptHistoryData getApplicationAttempt( + ApplicationAttemptId appAttemptId) throws IOException; + + /** + * This method returns {@link ContainerHistoryData} for specified + * {@link ContainerId}. + * + * @param {@link ContainerId} + * @return {@link ContainerHistoryData} for ContainerId + * @throws {@link IOException} + */ + ContainerHistoryData getContainer(ContainerId containerId) throws IOException; + + /** + * This method returns {@link ContainerHistoryData} for specified + * {@link ApplicationAttemptId}. + * + * @param {@link ApplicationAttemptId} + * @return {@link ContainerHistoryData} for ApplicationAttemptId + * @throws {@link IOException} + */ + ContainerHistoryData getAMContainer(ApplicationAttemptId appAttemptId) + throws IOException; + + /** + * This method returns Map{@link ContainerId,@link ContainerHistoryData} for + * specified {@link ApplicationAttemptId}. + * + * @param {@link ApplicationAttemptId} + * @return Map{@link ContainerId, @link ContainerHistoryData} for + * ApplicationAttemptId + * @throws {@link IOException} + */ + Map getContainers( + ApplicationAttemptId appAttemptId) throws IOException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java new file mode 100644 index 0000000..689510f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java @@ -0,0 +1,158 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.source.JvmMetrics; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.service.Service; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.ShutdownHookManager; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp; +import org.apache.hadoop.yarn.webapp.WebApp; +import org.apache.hadoop.yarn.webapp.WebApps; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils; + +import com.google.common.annotations.VisibleForTesting; + +/** + * History server that keeps track of all types of history in the cluster. + * Application specific history to start with. + */ +public class ApplicationHistoryServer extends CompositeService { + + public static final int SHUTDOWN_HOOK_PRIORITY = 30; + private static final Log LOG = LogFactory + .getLog(ApplicationHistoryServer.class); + + ApplicationHistoryClientService ahsClientService; + ApplicationHistoryManager historyManager; + private WebApp webApp; + + public ApplicationHistoryServer() { + super(ApplicationHistoryServer.class.getName()); + } + + @Override + protected void serviceInit(Configuration conf) throws Exception { + historyManager = createApplicationHistory(); + ahsClientService = createApplicationHistoryClientService(historyManager); + addService(ahsClientService); + addService((Service) historyManager); + super.serviceInit(conf); + } + + @Override + protected void serviceStart() throws Exception { + DefaultMetricsSystem.initialize("ApplicationHistoryServer"); + JvmMetrics.initSingleton("ApplicationHistoryServer", null); + + startWebApp(); + super.serviceStart(); + } + + @Override + protected void serviceStop() throws Exception { + if (webApp != null) { + webApp.stop(); + } + + DefaultMetricsSystem.shutdown(); + super.serviceStop(); + } + + @Private + @VisibleForTesting + public ApplicationHistoryClientService getClientService() { + return this.ahsClientService; + } + + protected ApplicationHistoryClientService createApplicationHistoryClientService( + ApplicationHistoryManager historyManager) { + return new ApplicationHistoryClientService(historyManager); + } + + protected ApplicationHistoryManager createApplicationHistory() { + return new ApplicationHistoryManagerImpl(); + } + + protected ApplicationHistoryManager getApplicationHistory() { + return this.historyManager; + } + + static ApplicationHistoryServer launchAppHistoryServer(String[] args) { + Thread + .setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler()); + StringUtils.startupShutdownMessage(ApplicationHistoryServer.class, args, + LOG); + ApplicationHistoryServer appHistoryServer = null; + try { + appHistoryServer = new ApplicationHistoryServer(); + ShutdownHookManager.get().addShutdownHook( + new CompositeServiceShutdownHook(appHistoryServer), + SHUTDOWN_HOOK_PRIORITY); + YarnConfiguration conf = new YarnConfiguration(); + appHistoryServer.init(conf); + appHistoryServer.start(); + } catch (Throwable t) { + LOG.fatal("Error starting ApplicationHistoryServer", t); + ExitUtil.terminate(-1, "Error starting ApplicationHistoryServer"); + } + return appHistoryServer; + } + + public static void main(String[] args) { + launchAppHistoryServer(args); + } + + protected ApplicationHistoryManager createApplicationHistoryManager( + Configuration conf) { + return new ApplicationHistoryManagerImpl(); + } + + protected void startWebApp() { + String bindAddress = WebAppUtils.getAHSWebAppURLWithoutScheme(getConfig()); + LOG.info("Instantiating AHSWebApp at " + bindAddress); + try { + webApp = WebApps + .$for("applicationhistory", ApplicationHistoryClientService.class, + ahsClientService, "ws") + .with(getConfig()) + .withHttpSpnegoPrincipalKey( + YarnConfiguration.AHS_WEBAPP_SPNEGO_USER_NAME_KEY) + .withHttpSpnegoKeytabKey( + YarnConfiguration.AHS_WEBAPP_SPNEGO_KEYTAB_FILE_KEY) + .at(bindAddress) + .start(new AHSWebApp(historyManager)); + } catch (Exception e) { + String msg = "AHSWebApp failed to start."; + LOG.error(msg, e); + throw new YarnRuntimeException(msg, e); + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryStore.java new file mode 100644 index 0000000..c26faef --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryStore.java @@ -0,0 +1,37 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.service.Service; + +/** + * This class is the abstract of the storage of the application history data. It + * is a {@link Service}, such that the implementation of this class can make use + * of the service life cycle to initialize and cleanup the storage. Users can + * access the storage via {@link ApplicationHistoryReader} and + * {@link ApplicationHistoryWriter} interfaces. + * + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public interface ApplicationHistoryStore extends Service, + ApplicationHistoryReader, ApplicationHistoryWriter { +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryWriter.java new file mode 100644 index 0000000..46e83a1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryWriter.java @@ -0,0 +1,110 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerStartData; + +/** + * It is the interface of writing the application history, exposing the methods + * of writing {@link ApplicationHistoryData}, + * {@link ApplicationAttemptHistoryData} and {@link ContainerHistoryData}. + */ +@Private +@Unstable +public interface ApplicationHistoryWriter { + + /** + * This method writes the information of RMApp that is available + * when it starts. + * + * @param appStart + * the record of the information of RMApp that is + * available when it starts + * @throws IOException + */ + void applicationStarted(ApplicationStartData appStart) throws IOException; + + /** + * This method writes the information of RMApp that is available + * when it finishes. + * + * @param appFinish + * the record of the information of RMApp that is + * available when it finishes + * @throws IOException + */ + void applicationFinished(ApplicationFinishData appFinish) throws IOException; + + /** + * This method writes the information of RMAppAttempt that is + * available when it starts. + * + * @param appAttemptStart + * the record of the information of RMAppAttempt that is + * available when it starts + * @throws IOException + */ + void applicationAttemptStarted( + ApplicationAttemptStartData appAttemptStart) throws IOException; + + /** + * This method writes the information of RMAppAttempt that is + * available when it finishes. + * + * @param appAttemptFinish + * the record of the information of RMAppAttempt that is + * available when it finishes + * @throws IOException + */ + void applicationAttemptFinished( + ApplicationAttemptFinishData appAttemptFinish) throws IOException; + + /** + * This method writes the information of RMContainer that is + * available when it starts. + * + * @param containerStart + * the record of the information of RMContainer that is + * available when it starts + * @throws IOException + */ + void containerStarted(ContainerStartData containerStart) throws IOException; + + /** + * This method writes the information of RMContainer that is + * available when it finishes. + * + * @param containerFinish + * the record of the information of RMContainer that is + * available when it finishes + * @throws IOException + */ + void containerFinished(ContainerFinishData containerFinish) + throws IOException; + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java new file mode 100644 index 0000000..b4d97f3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java @@ -0,0 +1,860 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.file.tfile.TFile; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationAttemptFinishDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationAttemptStartDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationFinishDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationStartDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ContainerFinishDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ContainerStartDataProto; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.impl.pb.ApplicationAttemptFinishDataPBImpl; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.impl.pb.ApplicationAttemptStartDataPBImpl; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.impl.pb.ApplicationFinishDataPBImpl; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.impl.pb.ApplicationStartDataPBImpl; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.impl.pb.ContainerFinishDataPBImpl; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.impl.pb.ContainerStartDataPBImpl; +import org.apache.hadoop.yarn.util.ConverterUtils; + +import com.google.protobuf.InvalidProtocolBufferException; + +/** + * File system implementation of {@link ApplicationHistoryStore}. In this + * implementation, one application will have just one file in the file system, + * which contains all the history data of one application, and its attempts and + * containers. {@link #applicationStarted(ApplicationStartData)} is supposed to + * be invoked first when writing any history data of one application and it will + * open a file, while {@link #applicationFinished(ApplicationFinishData)} is + * supposed to be last writing operation and will close the file. + */ +@Public +@Unstable +public class FileSystemApplicationHistoryStore extends AbstractService + implements ApplicationHistoryStore { + + private static final Log LOG = LogFactory + .getLog(FileSystemApplicationHistoryStore.class); + + private static final String ROOT_DIR_NAME = "ApplicationHistoryDataRoot"; + private static final int MIN_BLOCK_SIZE = 256 * 1024; + private static final String START_DATA_SUFFIX = "_start"; + private static final String FINISH_DATA_SUFFIX = "_finish"; + private static final FsPermission ROOT_DIR_UMASK = + FsPermission.createImmutable((short) 0740); + private static final FsPermission HISTORY_FILE_UMASK = + FsPermission.createImmutable((short) 0640); + + private FileSystem fs; + private Path rootDirPath; + + private ConcurrentMap outstandingWriters = + new ConcurrentHashMap(); + + public FileSystemApplicationHistoryStore() { + super(FileSystemApplicationHistoryStore.class.getName()); + } + + @Override + public void serviceInit(Configuration conf) throws Exception { + Path fsWorkingPath = new Path( + conf.get(YarnConfiguration.FS_HISTORY_STORE_URI)); + rootDirPath = new Path(fsWorkingPath, ROOT_DIR_NAME); + try { + fs = fsWorkingPath.getFileSystem(conf); + fs.mkdirs(rootDirPath); + fs.setPermission(rootDirPath, ROOT_DIR_UMASK); + } catch (IOException e) { + LOG.error("Error when initializing FileSystemHistoryStorage", e); + throw e; + } + super.serviceInit(conf); + } + + @Override + public void serviceStop() throws Exception { + try { + for (Entry entry : outstandingWriters + .entrySet()) { + entry.getValue().close(); + } + outstandingWriters.clear(); + } finally { + IOUtils.cleanup(LOG, fs); + } + super.serviceStop(); + } + + @Override + public ApplicationHistoryData getApplication(ApplicationId appId) + throws IOException { + HistoryFileReader hfReader = getHistoryFileReader(appId); + try { + boolean readStartData = false; + boolean readFinishData = false; + ApplicationHistoryData historyData = + ApplicationHistoryData.newInstance( + appId, null, null, null, null, Long.MIN_VALUE, Long.MIN_VALUE, + Long.MAX_VALUE, null, FinalApplicationStatus.UNDEFINED, null); + while ((!readStartData || !readFinishData) && hfReader.hasNext()) { + HistoryFileReader.Entry entry = hfReader.next(); + if (entry.key.id.equals(appId.toString())) { + if (entry.key.suffix.equals(START_DATA_SUFFIX)) { + ApplicationStartData startData = + parseApplicationStartData(entry.value); + mergeApplicationHistoryData(historyData, startData); + readStartData = true; + } else if (entry.key.suffix.equals(FINISH_DATA_SUFFIX)) { + ApplicationFinishData finishData = + parseApplicationFinishData(entry.value); + mergeApplicationHistoryData(historyData, finishData); + readFinishData = true; + } + } + } + if (!readStartData && !readFinishData) { + return null; + } + if (!readStartData) { + LOG.warn("Start information is missing for application " + appId); + } + if (!readFinishData) { + LOG.warn("Finish information is missing for application " + appId); + } + LOG.info("Completed reading history information of application " + appId); + return historyData; + } catch (IOException e) { + LOG.error("Error when reading history file of application " + appId); + throw e; + } finally { + hfReader.close(); + } + } + + @Override + public Map getAllApplications() + throws IOException { + Map historyDataMap = + new HashMap(); + FileStatus[] files = fs.listStatus(rootDirPath); + for (FileStatus file : files) { + ApplicationId appId = + ConverterUtils.toApplicationId(file.getPath().getName()); + try { + ApplicationHistoryData historyData = getApplication(appId); + if (historyData != null) { + historyDataMap.put(appId, historyData); + } + } catch (IOException e) { + // Eat the exception not to disturb the getting the next + // ApplicationHistoryData + LOG.error("History information of application " + appId + + " is not included into the result due to the exception", e); + } + } + return historyDataMap; + } + + @Override + public Map + getApplicationAttempts(ApplicationId appId) throws IOException { + Map historyDataMap = + new HashMap(); + Map> startFinshDataMap = + new HashMap>(); + HistoryFileReader hfReader = getHistoryFileReader(appId); + try { + while (hfReader.hasNext()) { + HistoryFileReader.Entry entry = hfReader.next(); + if (entry.key.id.startsWith(ConverterUtils.APPLICATION_ATTEMPT_PREFIX)) { + if (entry.key.suffix.equals(START_DATA_SUFFIX)) { + retrieveStartFinishData(appId, entry, startFinshDataMap, true); + } else if (entry.key.suffix.equals(FINISH_DATA_SUFFIX)) { + retrieveStartFinishData(appId, entry, startFinshDataMap, false); + } + } + } + LOG.info("Completed reading history information of all application" + + " attempts of application " + appId); + } catch (IOException e) { + LOG.info("Error when reading history information of some application" + + " attempts of application " + appId); + } finally { + hfReader.close(); + } + for (Map.Entry> entry : startFinshDataMap + .entrySet()) { + ApplicationAttemptHistoryData historyData = + ApplicationAttemptHistoryData.newInstance( + entry.getKey(), null, -1, null, null, null, + FinalApplicationStatus.UNDEFINED, null); + mergeApplicationAttemptHistoryData(historyData, + entry.getValue().startData); + mergeApplicationAttemptHistoryData(historyData, + entry.getValue().finishData); + historyDataMap.put(entry.getKey(), historyData); + } + return historyDataMap; + } + + private + void + retrieveStartFinishData( + ApplicationId appId, + HistoryFileReader.Entry entry, + Map> startFinshDataMap, + boolean start) throws IOException { + ApplicationAttemptId appAttemptId = + ConverterUtils.toApplicationAttemptId(entry.key.id); + if (appAttemptId.getApplicationId().equals(appId)) { + StartFinishDataPair pair = + startFinshDataMap.get(appAttemptId); + if (pair == null) { + pair = + new StartFinishDataPair(); + startFinshDataMap.put(appAttemptId, pair); + } + if (start) { + pair.startData = parseApplicationAttemptStartData(entry.value); + } else { + pair.finishData = parseApplicationAttemptFinishData(entry.value); + } + } + } + + @Override + public ApplicationAttemptHistoryData getApplicationAttempt( + ApplicationAttemptId appAttemptId) throws IOException { + HistoryFileReader hfReader = + getHistoryFileReader(appAttemptId.getApplicationId()); + try { + boolean readStartData = false; + boolean readFinishData = false; + ApplicationAttemptHistoryData historyData = + ApplicationAttemptHistoryData.newInstance( + appAttemptId, null, -1, null, null, null, + FinalApplicationStatus.UNDEFINED, null); + while ((!readStartData || !readFinishData) && hfReader.hasNext()) { + HistoryFileReader.Entry entry = hfReader.next(); + if (entry.key.id.equals(appAttemptId.toString())) { + if (entry.key.suffix.equals(START_DATA_SUFFIX)) { + ApplicationAttemptStartData startData = + parseApplicationAttemptStartData(entry.value); + mergeApplicationAttemptHistoryData(historyData, startData); + readStartData = true; + } else if (entry.key.suffix.equals(FINISH_DATA_SUFFIX)) { + ApplicationAttemptFinishData finishData = + parseApplicationAttemptFinishData(entry.value); + mergeApplicationAttemptHistoryData(historyData, finishData); + readFinishData = true; + } + } + } + if (!readStartData && !readFinishData) { + return null; + } + if (!readStartData) { + LOG.warn("Start information is missing for application attempt " + + appAttemptId); + } + if (!readFinishData) { + LOG.warn("Finish information is missing for application attempt " + + appAttemptId); + } + LOG.info("Completed reading history information of application attempt " + + appAttemptId); + return historyData; + } catch (IOException e) { + LOG.error("Error when reading history file of application attempt" + + appAttemptId); + throw e; + } finally { + hfReader.close(); + } + } + + @Override + public ContainerHistoryData getContainer(ContainerId containerId) + throws IOException { + HistoryFileReader hfReader = + getHistoryFileReader(containerId.getApplicationAttemptId() + .getApplicationId()); + try { + boolean readStartData = false; + boolean readFinishData = false; + ContainerHistoryData historyData = + ContainerHistoryData.newInstance(containerId, null, null, null, + Long.MIN_VALUE, Long.MAX_VALUE, null, null, Integer.MAX_VALUE, + null); + while ((!readStartData || !readFinishData) && hfReader.hasNext()) { + HistoryFileReader.Entry entry = hfReader.next(); + if (entry.key.id.equals(containerId.toString())) { + if (entry.key.suffix.equals(START_DATA_SUFFIX)) { + ContainerStartData startData = + parseContainerStartData(entry.value); + mergeContainerHistoryData(historyData, startData); + readStartData = true; + } else if (entry.key.suffix.equals(FINISH_DATA_SUFFIX)) { + ContainerFinishData finishData = + parseContainerFinishData(entry.value); + mergeContainerHistoryData(historyData, finishData); + readFinishData = true; + } + } + } + if (!readStartData && !readFinishData) { + return null; + } + if (!readStartData) { + LOG.warn("Start information is missing for container " + containerId); + } + if (!readFinishData) { + LOG.warn("Finish information is missing for container " + containerId); + } + LOG.info("Completed reading history information of container " + + containerId); + return historyData; + } catch (IOException e) { + LOG.error("Error when reading history file of container " + containerId); + throw e; + } finally { + hfReader.close(); + } + } + + @Override + public ContainerHistoryData getAMContainer(ApplicationAttemptId appAttemptId) + throws IOException { + ApplicationAttemptHistoryData attemptHistoryData = + getApplicationAttempt(appAttemptId); + if (attemptHistoryData == null + || attemptHistoryData.getMasterContainerId() == null) { + return null; + } + return getContainer(attemptHistoryData.getMasterContainerId()); + } + + @Override + public Map getContainers( + ApplicationAttemptId appAttemptId) throws IOException { + Map historyDataMap = + new HashMap(); + Map> startFinshDataMap = + new HashMap>(); + HistoryFileReader hfReader = + getHistoryFileReader(appAttemptId.getApplicationId()); + try { + while (hfReader.hasNext()) { + HistoryFileReader.Entry entry = hfReader.next(); + if (entry.key.id.startsWith(ConverterUtils.CONTAINER_PREFIX)) { + if (entry.key.suffix.equals(START_DATA_SUFFIX)) { + retrieveStartFinishData(appAttemptId, entry, startFinshDataMap, + true); + } else if (entry.key.suffix.equals(FINISH_DATA_SUFFIX)) { + retrieveStartFinishData(appAttemptId, entry, startFinshDataMap, + false); + } + } + } + LOG.info("Completed reading history information of all conatiners" + + " of application attempt " + appAttemptId); + } catch (IOException e) { + LOG.info("Error when reading history information of some containers" + + " of application attempt " + appAttemptId); + } finally { + hfReader.close(); + } + for (Map.Entry> entry : startFinshDataMap + .entrySet()) { + ContainerHistoryData historyData = + ContainerHistoryData.newInstance(entry.getKey(), null, null, null, + Long.MIN_VALUE, Long.MAX_VALUE, null, null, Integer.MAX_VALUE, + null); + mergeContainerHistoryData(historyData, entry.getValue().startData); + mergeContainerHistoryData(historyData, entry.getValue().finishData); + historyDataMap.put(entry.getKey(), historyData); + } + return historyDataMap; + } + + private + void + retrieveStartFinishData( + ApplicationAttemptId appAttemptId, + HistoryFileReader.Entry entry, + Map> startFinshDataMap, + boolean start) throws IOException { + ContainerId containerId = + ConverterUtils.toContainerId(entry.key.id); + if (containerId.getApplicationAttemptId().equals(appAttemptId)) { + StartFinishDataPair pair = + startFinshDataMap.get(containerId); + if (pair == null) { + pair = + new StartFinishDataPair(); + startFinshDataMap.put(containerId, pair); + } + if (start) { + pair.startData = parseContainerStartData(entry.value); + } else { + pair.finishData = parseContainerFinishData(entry.value); + } + } + } + + @Override + public void applicationStarted(ApplicationStartData appStart) + throws IOException { + HistoryFileWriter hfWriter = + outstandingWriters.get(appStart.getApplicationId()); + if (hfWriter == null) { + Path applicationHistoryFile = + new Path(rootDirPath, appStart.getApplicationId().toString()); + try { + hfWriter = new HistoryFileWriter(applicationHistoryFile); + LOG.info("Opened history file of application " + + appStart.getApplicationId()); + } catch (IOException e) { + LOG.error("Error when openning history file of application " + + appStart.getApplicationId()); + throw e; + } + outstandingWriters.put(appStart.getApplicationId(), hfWriter); + } else { + throw new IOException("History file of application " + + appStart.getApplicationId() + " is already opened"); + } + assert appStart instanceof ApplicationStartDataPBImpl; + try { + hfWriter.writeHistoryData(new HistoryDataKey(appStart.getApplicationId() + .toString(), START_DATA_SUFFIX), + ((ApplicationStartDataPBImpl) appStart) + .getProto().toByteArray()); + LOG.info("Start information of application " + + appStart.getApplicationId() + " is written"); + } catch (IOException e) { + LOG.error("Error when writing start information of application " + + appStart.getApplicationId()); + throw e; + } + } + + @Override + public void applicationFinished(ApplicationFinishData appFinish) + throws IOException { + HistoryFileWriter hfWriter = + getHistoryFileWriter(appFinish.getApplicationId()); + assert appFinish instanceof ApplicationFinishDataPBImpl; + try { + hfWriter.writeHistoryData( + new HistoryDataKey(appFinish.getApplicationId().toString(), + FINISH_DATA_SUFFIX), + ((ApplicationFinishDataPBImpl) appFinish).getProto().toByteArray()); + LOG.info("Finish information of application " + + appFinish.getApplicationId() + " is written"); + } catch (IOException e) { + LOG.error("Error when writing finish information of application " + + appFinish.getApplicationId()); + throw e; + } finally { + hfWriter.close(); + outstandingWriters.remove(appFinish.getApplicationId()); + } + } + + @Override + public void applicationAttemptStarted( + ApplicationAttemptStartData appAttemptStart) throws IOException { + HistoryFileWriter hfWriter = + getHistoryFileWriter(appAttemptStart.getApplicationAttemptId() + .getApplicationId()); + assert appAttemptStart instanceof ApplicationAttemptStartDataPBImpl; + try { + hfWriter.writeHistoryData( + new HistoryDataKey(appAttemptStart.getApplicationAttemptId() + .toString(), + START_DATA_SUFFIX), + ((ApplicationAttemptStartDataPBImpl) appAttemptStart).getProto() + .toByteArray()); + LOG.info("Start information of application attempt " + + appAttemptStart.getApplicationAttemptId() + " is written"); + } catch (IOException e) { + LOG.error("Error when writing start information of application attempt " + + appAttemptStart.getApplicationAttemptId()); + throw e; + } + } + + @Override + public void applicationAttemptFinished( + ApplicationAttemptFinishData appAttemptFinish) throws IOException { + HistoryFileWriter hfWriter = + getHistoryFileWriter(appAttemptFinish.getApplicationAttemptId() + .getApplicationId()); + assert appAttemptFinish instanceof ApplicationAttemptFinishDataPBImpl; + try { + hfWriter.writeHistoryData( + new HistoryDataKey(appAttemptFinish.getApplicationAttemptId() + .toString(), + FINISH_DATA_SUFFIX), + ((ApplicationAttemptFinishDataPBImpl) appAttemptFinish).getProto() + .toByteArray()); + LOG.info("Finish information of application attempt " + + appAttemptFinish.getApplicationAttemptId() + " is written"); + } catch (IOException e) { + LOG.error("Error when writing finish information of application attempt " + + appAttemptFinish.getApplicationAttemptId()); + throw e; + } + } + + @Override + public void containerStarted(ContainerStartData containerStart) + throws IOException { + HistoryFileWriter hfWriter = + getHistoryFileWriter(containerStart.getContainerId() + .getApplicationAttemptId() + .getApplicationId()); + assert containerStart instanceof ContainerStartDataPBImpl; + try { + hfWriter.writeHistoryData( + new HistoryDataKey(containerStart.getContainerId().toString(), + START_DATA_SUFFIX), + ((ContainerStartDataPBImpl) containerStart).getProto().toByteArray()); + LOG.info("Start information of container " + + containerStart.getContainerId() + " is written"); + } catch (IOException e) { + LOG.error("Error when writing start information of container " + + containerStart.getContainerId()); + throw e; + } + } + + @Override + public void containerFinished(ContainerFinishData containerFinish) + throws IOException { + HistoryFileWriter hfWriter = + getHistoryFileWriter(containerFinish.getContainerId() + .getApplicationAttemptId().getApplicationId()); + assert containerFinish instanceof ContainerFinishDataPBImpl; + try { + hfWriter.writeHistoryData( + new HistoryDataKey(containerFinish.getContainerId().toString(), + FINISH_DATA_SUFFIX), + ((ContainerFinishDataPBImpl) containerFinish).getProto() + .toByteArray()); + LOG.info("Finish information of container " + + containerFinish.getContainerId() + " is written"); + } catch (IOException e) { + LOG.error("Error when writing finish information of container " + + containerFinish.getContainerId()); + } + } + + private static ApplicationStartData parseApplicationStartData(byte[] value) + throws InvalidProtocolBufferException { + return new ApplicationStartDataPBImpl( + ApplicationStartDataProto.parseFrom(value)); + } + + private static ApplicationFinishData parseApplicationFinishData(byte[] value) + throws InvalidProtocolBufferException { + return new ApplicationFinishDataPBImpl( + ApplicationFinishDataProto.parseFrom(value)); + } + + private static ApplicationAttemptStartData parseApplicationAttemptStartData( + byte[] value) throws InvalidProtocolBufferException { + return new ApplicationAttemptStartDataPBImpl( + ApplicationAttemptStartDataProto.parseFrom(value)); + } + + private static ApplicationAttemptFinishData + parseApplicationAttemptFinishData( + byte[] value) throws InvalidProtocolBufferException { + return new ApplicationAttemptFinishDataPBImpl( + ApplicationAttemptFinishDataProto.parseFrom(value)); + } + + private static ContainerStartData parseContainerStartData(byte[] value) + throws InvalidProtocolBufferException { + return new ContainerStartDataPBImpl( + ContainerStartDataProto.parseFrom(value)); + } + + private static ContainerFinishData parseContainerFinishData(byte[] value) + throws InvalidProtocolBufferException { + return new ContainerFinishDataPBImpl( + ContainerFinishDataProto.parseFrom(value)); + } + + private static void mergeApplicationHistoryData( + ApplicationHistoryData historyData, + ApplicationStartData startData) { + historyData.setApplicationName(startData.getApplicationName()); + historyData.setApplicationType(startData.getApplicationType()); + historyData.setQueue(startData.getQueue()); + historyData.setUser(startData.getUser()); + historyData.setSubmitTime(startData.getSubmitTime()); + historyData.setStartTime(startData.getStartTime()); + } + + private static void mergeApplicationHistoryData( + ApplicationHistoryData historyData, + ApplicationFinishData finishData) { + historyData.setFinishTime(finishData.getFinishTime()); + historyData.setDiagnosticsInfo(finishData.getDiagnosticsInfo()); + historyData.setFinalApplicationStatus(finishData + .getFinalApplicationStatus()); + historyData.setYarnApplicationState(finishData.getYarnApplicationState()); + } + + private static void mergeApplicationAttemptHistoryData( + ApplicationAttemptHistoryData historyData, + ApplicationAttemptStartData startData) { + historyData.setHost(startData.getHost()); + historyData.setRPCPort(startData.getRPCPort()); + historyData.setMasterContainerId(startData.getMasterContainerId()); + } + + private static void mergeApplicationAttemptHistoryData( + ApplicationAttemptHistoryData historyData, + ApplicationAttemptFinishData finishData) { + historyData.setDiagnosticsInfo(finishData.getDiagnosticsInfo()); + historyData.setTrackingURL(finishData.getTrackingURL()); + historyData.setFinalApplicationStatus(finishData + .getFinalApplicationStatus()); + historyData.setYarnApplicationAttemptState(finishData + .getYarnApplicationAttemptState()); + } + + private static void mergeContainerHistoryData( + ContainerHistoryData historyData, ContainerStartData startData) { + historyData.setAllocatedResource(startData.getAllocatedResource()); + historyData.setAssignedNode(startData.getAssignedNode()); + historyData.setPriority(startData.getPriority()); + historyData.setStartTime(startData.getStartTime()); + } + + private static void mergeContainerHistoryData( + ContainerHistoryData historyData, ContainerFinishData finishData) { + historyData.setFinishTime(finishData.getFinishTime()); + historyData.setDiagnosticsInfo(finishData.getDiagnosticsInfo()); + historyData.setLogURL(finishData.getLogURL()); + historyData.setContainerExitStatus(finishData + .getContainerExitStatus()); + historyData.setContainerState(finishData.getContainerState()); + } + + private HistoryFileWriter getHistoryFileWriter(ApplicationId appId) + throws IOException { + HistoryFileWriter hfWriter = outstandingWriters.get(appId); + if (hfWriter == null) { + throw new IOException("History file of application " + appId + + " is not opened"); + } + return hfWriter; + } + + private HistoryFileReader getHistoryFileReader(ApplicationId appId) + throws IOException { + Path applicationHistoryFile = new Path(rootDirPath, appId.toString()); + if (!fs.exists(applicationHistoryFile)) { + throw new IOException("History file for application " + appId + + " is not found"); + } + // The history file is still under writing + if (outstandingWriters.containsKey(appId)) { + throw new IOException("History file for application " + appId + + " is under writing"); + } + return new HistoryFileReader(applicationHistoryFile); + } + + private class HistoryFileReader { + + private class Entry { + + private HistoryDataKey key; + private byte[] value; + + public Entry(HistoryDataKey key, byte[] value) { + this.key = key; + this.value = value; + } + } + + private FSDataInputStream fsdis; + private TFile.Reader reader; + private TFile.Reader.Scanner scanner; + + public HistoryFileReader(Path historyFile) throws IOException { + FSDataInputStream fsdis = fs.open(historyFile); + reader = + new TFile.Reader(fsdis, fs.getFileStatus(historyFile).getLen(), + getConfig()); + reset(); + } + + public boolean hasNext() { + return !scanner.atEnd(); + } + + public Entry next() throws IOException { + TFile.Reader.Scanner.Entry entry = scanner.entry(); + DataInputStream dis = entry.getKeyStream(); + HistoryDataKey key = new HistoryDataKey(); + key.readFields(dis); + dis = entry.getValueStream(); + byte[] value = new byte[entry.getValueLength()]; + dis.read(value); + scanner.advance(); + return new Entry(key, value); + } + + public void reset() throws IOException { + IOUtils.cleanup(LOG, scanner); + scanner = reader.createScanner(); + } + + public void close() { + IOUtils.cleanup(LOG, scanner, reader, fsdis); + } + + } + + private class HistoryFileWriter { + + private FSDataOutputStream fsdos; + private TFile.Writer writer; + + public HistoryFileWriter(Path historyFile) + throws IOException { + if (fs.exists(historyFile)) { + fsdos = fs.append(historyFile); + } else { + fsdos = fs.create(historyFile); + } + fs.setPermission(historyFile, HISTORY_FILE_UMASK); + writer = + new TFile.Writer(fsdos, MIN_BLOCK_SIZE, getConfig().get( + YarnConfiguration.FS_HISTORY_STORE_COMPRESSION_TYPE, + YarnConfiguration.DEFAULT_FS_HISTORY_STORE_COMPRESSION_TYPE), + null, getConfig()); + } + + public synchronized void close() { + IOUtils.cleanup(LOG, writer, fsdos); + } + + public synchronized void writeHistoryData(HistoryDataKey key, byte[] value) + throws IOException { + DataOutputStream dos = null; + try { + dos = writer.prepareAppendKey(-1); + key.write(dos); + } finally { + IOUtils.cleanup(LOG, dos); + } + try { + dos = writer.prepareAppendValue(value.length); + dos.write(value); + } finally { + IOUtils.cleanup(LOG, dos); + } + } + + } + + private static class HistoryDataKey implements Writable { + + private String id; + + private String suffix; + + public HistoryDataKey() { + this(null, null); + } + + public HistoryDataKey(String id, String suffix) { + this.id = id; + this.suffix = suffix; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeUTF(id); + out.writeUTF(suffix); + } + + @Override + public void readFields(DataInput in) throws IOException { + id = in.readUTF(); + suffix = in.readUTF(); + } + + } + + private static class StartFinishDataPair { + + private S startData; + private F finishData; + + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/MemoryApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/MemoryApplicationHistoryStore.java new file mode 100644 index 0000000..309ad4b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/MemoryApplicationHistoryStore.java @@ -0,0 +1,284 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerStartData; + +/** + * In-memory implementation of {@link ApplicationHistoryStore}. + * This implementation is for test purpose only. If users improperly instantiate + * it, they may encounter reading and writing history data in different memory + * store. + * + */ +@Private +@Unstable +public class MemoryApplicationHistoryStore extends AbstractService + implements ApplicationHistoryStore { + + private final ConcurrentMap applicationData = + new ConcurrentHashMap(); + private final ConcurrentMap> applicationAttemptData = + new ConcurrentHashMap>(); + private final ConcurrentMap> containerData = + new ConcurrentHashMap>(); + + public MemoryApplicationHistoryStore() { + super(MemoryApplicationHistoryStore.class.getName()); + } + + @Override + public Map getAllApplications() { + return new HashMap( + applicationData); + } + + @Override + public ApplicationHistoryData getApplication(ApplicationId appId) { + return applicationData.get(appId); + } + + @Override + public Map + getApplicationAttempts( + ApplicationId appId) { + ConcurrentMap subMap = + applicationAttemptData.get(appId); + if (subMap == null) { + return Collections.emptyMap(); + } else { + return new HashMap(subMap); + } + } + + @Override + public ApplicationAttemptHistoryData getApplicationAttempt( + ApplicationAttemptId appAttemptId) { + ConcurrentMap subMap = + applicationAttemptData.get(appAttemptId.getApplicationId()); + if (subMap == null) { + return null; + } else { + return subMap.get(appAttemptId); + } + } + + @Override + public ContainerHistoryData getAMContainer( + ApplicationAttemptId appAttemptId) { + ApplicationAttemptHistoryData appAttempt = + getApplicationAttempt(appAttemptId); + if (appAttempt == null || appAttempt.getMasterContainerId() == null) { + return null; + } else { + return getContainer(appAttempt.getMasterContainerId()); + } + } + + @Override + public ContainerHistoryData getContainer(ContainerId containerId) { + Map subMap = + containerData.get(containerId.getApplicationAttemptId()); + if (subMap == null) { + return null; + } else { + return subMap.get(containerId); + } + } + + @Override + public Map getContainers( + ApplicationAttemptId appAttemptId) throws IOException { + ConcurrentMap subMap = + containerData.get(appAttemptId); + if (subMap == null) { + return Collections.emptyMap(); + } else { + return new HashMap(subMap); + } + } + + @Override + public void applicationStarted(ApplicationStartData appStart) + throws IOException { + ApplicationHistoryData oldData = + applicationData.putIfAbsent(appStart.getApplicationId(), + ApplicationHistoryData.newInstance( + appStart.getApplicationId(), + appStart.getApplicationName(), + appStart.getApplicationType(), + appStart.getQueue(), + appStart.getUser(), + appStart.getSubmitTime(), + appStart.getStartTime(), + Long.MAX_VALUE, null, null, null)); + if (oldData != null) { + throw new IOException("The start information of application " + + appStart.getApplicationId() + " is already stored."); + } + } + + @Override + public void applicationFinished(ApplicationFinishData appFinish) + throws IOException { + ApplicationHistoryData data = + applicationData.get(appFinish.getApplicationId()); + if (data == null) { + throw new IOException("The finish information of application " + + appFinish.getApplicationId() + " is stored before the start" + + " information."); + } + // Make the assumption that YarnApplicationState should not be null if + // the finish information is already recorded + if (data.getYarnApplicationState() != null) { + throw new IOException("The finish information of application " + + appFinish.getApplicationId() + " is already stored."); + } + data.setFinishTime(appFinish.getFinishTime()); + data.setDiagnosticsInfo(appFinish.getDiagnosticsInfo()); + data.setFinalApplicationStatus(appFinish.getFinalApplicationStatus()); + data.setYarnApplicationState(appFinish.getYarnApplicationState()); + } + + @Override + public void applicationAttemptStarted( + ApplicationAttemptStartData appAttemptStart) throws IOException { + ConcurrentMap subMap = + getSubMap(appAttemptStart.getApplicationAttemptId().getApplicationId()); + ApplicationAttemptHistoryData oldData = subMap.putIfAbsent( + appAttemptStart.getApplicationAttemptId(), + ApplicationAttemptHistoryData.newInstance( + appAttemptStart.getApplicationAttemptId(), + appAttemptStart.getHost(), + appAttemptStart.getRPCPort(), + appAttemptStart.getMasterContainerId(), + null, null, null, null)); + if (oldData != null) { + throw new IOException("The start information of application attempt " + + appAttemptStart.getApplicationAttemptId() + + " is already stored."); + } + } + + @Override + public void applicationAttemptFinished( + ApplicationAttemptFinishData appAttemptFinish) throws IOException { + ConcurrentMap subMap = + getSubMap(appAttemptFinish.getApplicationAttemptId().getApplicationId()); + ApplicationAttemptHistoryData data = + subMap.get(appAttemptFinish.getApplicationAttemptId()); + if (data == null) { + throw new IOException("The finish information of application attempt " + + appAttemptFinish.getApplicationAttemptId() + " is stored before" + + " the start information."); + } + // Make the assumption that YarnApplicationAttemptState should not be null + // if the finish information is already recorded + if (data.getYarnApplicationAttemptState() != null) { + throw new IOException("The finish information of application attempt " + + appAttemptFinish.getApplicationAttemptId() + + " is already stored."); + } + data.setTrackingURL(appAttemptFinish.getTrackingURL()); + data.setDiagnosticsInfo(appAttemptFinish.getDiagnosticsInfo()); + data.setFinalApplicationStatus(appAttemptFinish.getFinalApplicationStatus()); + data.setYarnApplicationAttemptState(appAttemptFinish.getYarnApplicationAttemptState()); + } + + private ConcurrentMap + getSubMap(ApplicationId appId) { + applicationAttemptData.putIfAbsent(appId, + new ConcurrentHashMap()); + return applicationAttemptData.get(appId); + } + + @Override + public void containerStarted(ContainerStartData containerStart) + throws IOException { + ConcurrentMap subMap = + getSubMap(containerStart.getContainerId().getApplicationAttemptId()); + ContainerHistoryData oldData = subMap.putIfAbsent( + containerStart.getContainerId(), + ContainerHistoryData.newInstance( + containerStart.getContainerId(), + containerStart.getAllocatedResource(), + containerStart.getAssignedNode(), + containerStart.getPriority(), + containerStart.getStartTime(), + Long.MAX_VALUE, null, null, Integer.MAX_VALUE, null)); + if (oldData != null) { + throw new IOException("The start information of container " + + containerStart.getContainerId() + " is already stored."); + } + } + + @Override + public void containerFinished(ContainerFinishData containerFinish) + throws IOException { + ConcurrentMap subMap = + getSubMap(containerFinish.getContainerId().getApplicationAttemptId()); + ContainerHistoryData data = subMap.get(containerFinish.getContainerId()); + if (data == null) { + throw new IOException("The finish information of container " + + containerFinish.getContainerId() + " is stored before" + + " the start information."); + } + // Make the assumption that ContainerState should not be null if + // the finish information is already recorded + if (data.getContainerState() != null) { + throw new IOException("The finish information of container " + + containerFinish.getContainerId() + " is already stored."); + } + data.setFinishTime(containerFinish.getFinishTime()); + data.setDiagnosticsInfo(containerFinish.getDiagnosticsInfo()); + data.setLogURL(containerFinish.getLogURL()); + data.setContainerExitStatus(containerFinish.getContainerExitStatus()); + data.setContainerState(containerFinish.getContainerState()); + } + + private ConcurrentMap getSubMap( + ApplicationAttemptId appAttemptId) { + containerData.putIfAbsent(appAttemptId, + new ConcurrentHashMap()); + return containerData.get(appAttemptId); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/NullApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/NullApplicationHistoryStore.java new file mode 100644 index 0000000..3660c10 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/NullApplicationHistoryStore.java @@ -0,0 +1,127 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerStartData; + +/** + * Dummy implementation of {@link ApplicationHistoryStore}. If this + * implementation is used, no history data will be persisted. + * + */ +@Unstable +@Private +public class NullApplicationHistoryStore extends AbstractService implements + ApplicationHistoryStore { + + public NullApplicationHistoryStore() { + super(NullApplicationHistoryStore.class.getName()); + } + + @Override + public void applicationStarted(ApplicationStartData appStart) + throws IOException { + } + + @Override + public void applicationFinished(ApplicationFinishData appFinish) + throws IOException { + } + + @Override + public void applicationAttemptStarted( + ApplicationAttemptStartData appAttemptStart) throws IOException { + } + + @Override + public void applicationAttemptFinished( + ApplicationAttemptFinishData appAttemptFinish) throws IOException { + } + + @Override + public void containerStarted(ContainerStartData containerStart) + throws IOException { + } + + @Override + public void containerFinished(ContainerFinishData containerFinish) + throws IOException { + } + + @Override + public ApplicationHistoryData getApplication(ApplicationId appId) + throws IOException { + return null; + } + + @Override + public Map getAllApplications() + throws IOException { + return Collections.emptyMap(); + } + + @Override + public Map + getApplicationAttempts(ApplicationId appId) throws IOException { + return Collections.emptyMap(); + } + + @Override + public ApplicationAttemptHistoryData getApplicationAttempt( + ApplicationAttemptId appAttemptId) throws IOException { + return null; + } + + @Override + public ContainerHistoryData getContainer(ContainerId containerId) + throws IOException { + return null; + } + + @Override + public ContainerHistoryData getAMContainer(ApplicationAttemptId appAttemptId) + throws IOException { + return null; + } + + @Override + public Map getContainers( + ApplicationAttemptId appAttemptId) throws IOException { + return Collections.emptyMap(); + } + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationAttemptFinishData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationAttemptFinishData.java new file mode 100644 index 0000000..7ba51af --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationAttemptFinishData.java @@ -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.hadoop.yarn.server.applicationhistoryservice.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; +import org.apache.hadoop.yarn.util.Records; + +/** + * The class contains the fields that can be determined when + * RMAppAttempt finishes, and that need to be stored persistently. + */ +@Public +@Unstable +public abstract class ApplicationAttemptFinishData { + + @Public + @Unstable + public static ApplicationAttemptFinishData newInstance( + ApplicationAttemptId appAttemptId, String diagnosticsInfo, + String trackingURL, FinalApplicationStatus finalApplicationStatus, + YarnApplicationAttemptState yarnApplicationAttemptState) { + ApplicationAttemptFinishData appAttemptFD = + Records.newRecord(ApplicationAttemptFinishData.class); + appAttemptFD.setApplicationAttemptId(appAttemptId); + appAttemptFD.setDiagnosticsInfo(diagnosticsInfo); + appAttemptFD.setTrackingURL(trackingURL); + appAttemptFD.setFinalApplicationStatus(finalApplicationStatus); + appAttemptFD.setYarnApplicationAttemptState(yarnApplicationAttemptState); + return appAttemptFD; + } + + @Public + @Unstable + public abstract ApplicationAttemptId getApplicationAttemptId(); + + @Public + @Unstable + public abstract void setApplicationAttemptId( + ApplicationAttemptId applicationAttemptId); + + @Public + @Unstable + public abstract String getTrackingURL(); + + @Public + @Unstable + public abstract void setTrackingURL(String trackingURL); + + @Public + @Unstable + public abstract String getDiagnosticsInfo(); + + @Public + @Unstable + public abstract void setDiagnosticsInfo(String diagnosticsInfo); + + @Public + @Unstable + public abstract FinalApplicationStatus getFinalApplicationStatus(); + + @Public + @Unstable + public abstract void setFinalApplicationStatus( + FinalApplicationStatus finalApplicationStatus); + + @Public + @Unstable + public abstract YarnApplicationAttemptState getYarnApplicationAttemptState(); + + @Public + @Unstable + public abstract void setYarnApplicationAttemptState( + YarnApplicationAttemptState yarnApplicationAttemptState); + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationAttemptHistoryData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationAttemptHistoryData.java new file mode 100644 index 0000000..deffceb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationAttemptHistoryData.java @@ -0,0 +1,172 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; + +/** + * The class contains all the fields that are stored persistently for + * RMAppAttempt. + */ +@Public +@Unstable +public class ApplicationAttemptHistoryData { + + private ApplicationAttemptId applicationAttemptId; + + private String host; + + private int rpcPort; + + private String trackingURL; + + private String diagnosticsInfo; + + private FinalApplicationStatus finalApplicationStatus; + + private ContainerId masterContainerId; + + private YarnApplicationAttemptState yarnApplicationAttemptState; + + @Public + @Unstable + public static ApplicationAttemptHistoryData newInstance( + ApplicationAttemptId appAttemptId, String host, int rpcPort, + ContainerId masterContainerId, String diagnosticsInfo, + String trackingURL, + FinalApplicationStatus finalApplicationStatus, + YarnApplicationAttemptState yarnApplicationAttemptState) { + ApplicationAttemptHistoryData appAttemptHD = + new ApplicationAttemptHistoryData(); + appAttemptHD.setApplicationAttemptId(appAttemptId); + appAttemptHD.setHost(host); + appAttemptHD.setRPCPort(rpcPort); + appAttemptHD.setMasterContainerId(masterContainerId); + appAttemptHD.setDiagnosticsInfo(diagnosticsInfo); + appAttemptHD.setTrackingURL(trackingURL); + appAttemptHD.setFinalApplicationStatus(finalApplicationStatus); + appAttemptHD.setYarnApplicationAttemptState(yarnApplicationAttemptState); + return appAttemptHD; + } + + @Public + @Unstable + public ApplicationAttemptId getApplicationAttemptId() { + return applicationAttemptId; + } + + @Public + @Unstable + public void setApplicationAttemptId( + ApplicationAttemptId applicationAttemptId) { + this.applicationAttemptId = applicationAttemptId; + } + + @Public + @Unstable + public String getHost() { + return host; + } + + @Public + @Unstable + public void setHost(String host) { + this.host = host; + } + + @Public + @Unstable + public int getRPCPort() { + return rpcPort; + } + + @Public + @Unstable + public void setRPCPort(int rpcPort) { + this.rpcPort = rpcPort; + } + + @Public + @Unstable + public String getTrackingURL() { + return trackingURL; + } + + @Public + @Unstable + public void setTrackingURL(String trackingURL) { + this.trackingURL = trackingURL; + } + + @Public + @Unstable + public String getDiagnosticsInfo() { + return diagnosticsInfo; + } + + @Public + @Unstable + public void setDiagnosticsInfo(String diagnosticsInfo) { + this.diagnosticsInfo = diagnosticsInfo; + } + + @Public + @Unstable + public FinalApplicationStatus getFinalApplicationStatus() { + return finalApplicationStatus; + } + + @Public + @Unstable + public void setFinalApplicationStatus( + FinalApplicationStatus finalApplicationStatus) { + this.finalApplicationStatus = finalApplicationStatus; + } + + @Public + @Unstable + public ContainerId getMasterContainerId() { + return masterContainerId; + } + + @Public + @Unstable + public void setMasterContainerId(ContainerId masterContainerId) { + this.masterContainerId = masterContainerId; + } + + @Public + @Unstable + public YarnApplicationAttemptState getYarnApplicationAttemptState() { + return yarnApplicationAttemptState; + } + + @Public + @Unstable + public void setYarnApplicationAttemptState( + YarnApplicationAttemptState yarnApplicationAttemptState) { + this.yarnApplicationAttemptState = yarnApplicationAttemptState; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationAttemptStartData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationAttemptStartData.java new file mode 100644 index 0000000..7ca43fa --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationAttemptStartData.java @@ -0,0 +1,82 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.util.Records; + +/** + * The class contains the fields that can be determined when + * RMAppAttempt starts, and that need to be stored persistently. + */ +@Public +@Unstable +public abstract class ApplicationAttemptStartData { + + @Public + @Unstable + public static ApplicationAttemptStartData newInstance( + ApplicationAttemptId appAttemptId, String host, int rpcPort, + ContainerId masterContainerId) { + ApplicationAttemptStartData appAttemptSD = + Records.newRecord(ApplicationAttemptStartData.class); + appAttemptSD.setApplicationAttemptId(appAttemptId); + appAttemptSD.setHost(host); + appAttemptSD.setRPCPort(rpcPort); + appAttemptSD.setMasterContainerId(masterContainerId); + return appAttemptSD; + } + + @Public + @Unstable + public abstract ApplicationAttemptId getApplicationAttemptId(); + + @Public + @Unstable + public abstract void setApplicationAttemptId( + ApplicationAttemptId applicationAttemptId); + + @Public + @Unstable + public abstract String getHost(); + + @Public + @Unstable + public abstract void setHost(String host); + + @Public + @Unstable + public abstract int getRPCPort(); + + @Public + @Unstable + public abstract void setRPCPort(int rpcPort); + + @Public + @Unstable + public abstract ContainerId getMasterContainerId(); + + @Public + @Unstable + public abstract void setMasterContainerId(ContainerId masterContainerId); + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationFinishData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationFinishData.java new file mode 100644 index 0000000..e94bfdd --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationFinishData.java @@ -0,0 +1,94 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.util.Records; + +/** + * The class contains the fields that can be determined when + * RMApp finishes, and that need to be stored persistently. + */ +@Public +@Unstable +public abstract class ApplicationFinishData { + + @Public + @Unstable + public static ApplicationFinishData newInstance(ApplicationId applicationId, + long finishTime, String diagnosticsInfo, + FinalApplicationStatus finalApplicationStatus, + YarnApplicationState yarnApplicationState) { + ApplicationFinishData appFD = + Records.newRecord(ApplicationFinishData.class); + appFD.setApplicationId(applicationId); + appFD.setFinishTime(finishTime); + appFD.setDiagnosticsInfo(diagnosticsInfo); + appFD.setFinalApplicationStatus(finalApplicationStatus); + appFD.setYarnApplicationState(yarnApplicationState); + return appFD; + } + + @Public + @Unstable + public abstract ApplicationId getApplicationId(); + + @Public + @Unstable + public abstract void setApplicationId(ApplicationId applicationId); + + @Public + @Unstable + public abstract long getFinishTime(); + + @Public + @Unstable + public abstract void setFinishTime(long finishTime); + + @Public + @Unstable + public abstract String getDiagnosticsInfo(); + + @Public + @Unstable + public abstract void setDiagnosticsInfo(String diagnosticsInfo); + + @Public + @Unstable + public abstract FinalApplicationStatus getFinalApplicationStatus(); + + @Public + @Unstable + public abstract void setFinalApplicationStatus( + FinalApplicationStatus finalApplicationStatus); + + @Public + @Unstable + public abstract YarnApplicationState getYarnApplicationState(); + + @Public + @Unstable + public abstract void setYarnApplicationState( + YarnApplicationState yarnApplicationState); + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationHistoryData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationHistoryData.java new file mode 100644 index 0000000..905b691 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationHistoryData.java @@ -0,0 +1,213 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; + +/** + * The class contains all the fields that are stored persistently for + * RMApp. + */ +@Public +@Unstable +public class ApplicationHistoryData { + + private ApplicationId applicationId; + + private String applicationName; + + private String applicationType; + + private String user; + + private String queue; + + private long submitTime; + + private long startTime; + + private long finishTime; + + private String diagnosticsInfo; + + private FinalApplicationStatus finalApplicationStatus; + + private YarnApplicationState yarnApplicationState; + + @Public + @Unstable + public static ApplicationHistoryData newInstance(ApplicationId applicationId, + String applicationName, String applicationType, String queue, + String user, long submitTime, long startTime, long finishTime, + String diagnosticsInfo, FinalApplicationStatus finalApplicationStatus, + YarnApplicationState yarnApplicationState) { + ApplicationHistoryData appHD = new ApplicationHistoryData(); + appHD.setApplicationId(applicationId); + appHD.setApplicationName(applicationName); + appHD.setApplicationType(applicationType); + appHD.setQueue(queue); + appHD.setUser(user); + appHD.setSubmitTime(submitTime); + appHD.setStartTime(startTime); + appHD.setFinishTime(finishTime); + appHD.setDiagnosticsInfo(diagnosticsInfo); + appHD.setFinalApplicationStatus(finalApplicationStatus); + appHD.setYarnApplicationState(yarnApplicationState); + return appHD; + } + + @Public + @Unstable + public ApplicationId getApplicationId() { + return applicationId; + } + + @Public + @Unstable + public void setApplicationId(ApplicationId applicationId) { + this.applicationId = applicationId; + } + + @Public + @Unstable + public String getApplicationName() { + return applicationName; + } + + @Public + @Unstable + public void setApplicationName(String applicationName) { + this.applicationName = applicationName; + } + + @Public + @Unstable + public String getApplicationType() { + return applicationType; + } + + @Public + @Unstable + public void setApplicationType(String applicationType) { + this.applicationType = applicationType; + } + + @Public + @Unstable + public String getUser() { + return user; + } + + @Public + @Unstable + public void setUser(String user) { + this.user = user; + } + + @Public + @Unstable + public String getQueue() { + return queue; + } + + @Public + @Unstable + public void setQueue(String queue) { + this.queue = queue; + } + + @Public + @Unstable + public long getSubmitTime() { + return submitTime; + } + + @Public + @Unstable + public void setSubmitTime(long submitTime) { + this.submitTime = submitTime; + } + + @Public + @Unstable + public long getStartTime() { + return startTime; + } + + @Public + @Unstable + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + @Public + @Unstable + public long getFinishTime() { + return finishTime; + } + + @Public + @Unstable + public void setFinishTime(long finishTime) { + this.finishTime = finishTime; + } + + @Public + @Unstable + public String getDiagnosticsInfo() { + return diagnosticsInfo; + } + + @Public + @Unstable + public void setDiagnosticsInfo(String diagnosticsInfo) { + this.diagnosticsInfo = diagnosticsInfo; + } + + @Public + @Unstable + public FinalApplicationStatus getFinalApplicationStatus() { + return finalApplicationStatus; + } + + @Public + @Unstable + public void setFinalApplicationStatus( + FinalApplicationStatus finalApplicationStatus) { + this.finalApplicationStatus = finalApplicationStatus; + } + + @Public + @Unstable + public YarnApplicationState getYarnApplicationState() { + return this.yarnApplicationState; + } + + @Public + @Unstable + public void setYarnApplicationState( + YarnApplicationState yarnApplicationState) { + this.yarnApplicationState = yarnApplicationState; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationStartData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationStartData.java new file mode 100644 index 0000000..021a061 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ApplicationStartData.java @@ -0,0 +1,108 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.util.Records; + +/** + * The class contains the fields that can be determined when + * RMApp starts, and that need to be stored persistently. + */ +@Public +@Unstable +public abstract class ApplicationStartData { + + @Public + @Unstable + public static ApplicationStartData newInstance( + ApplicationId applicationId, String applicationName, + String applicationType, String queue, String user, + long submitTime, long startTime) { + ApplicationStartData appSD = + Records.newRecord(ApplicationStartData.class); + appSD.setApplicationId(applicationId); + appSD.setApplicationName(applicationName); + appSD.setApplicationType(applicationType); + appSD.setQueue(queue); + appSD.setUser(user); + appSD.setSubmitTime(submitTime); + appSD.setStartTime(startTime); + return appSD; + } + + @Public + @Unstable + public abstract ApplicationId getApplicationId(); + + @Public + @Unstable + public abstract void setApplicationId(ApplicationId applicationId); + + @Public + @Unstable + public abstract String getApplicationName(); + + @Public + @Unstable + public abstract void setApplicationName(String applicationName); + + @Public + @Unstable + public abstract String getApplicationType(); + + @Public + @Unstable + public abstract void setApplicationType(String applicationType); + + @Public + @Unstable + public abstract String getUser(); + + @Public + @Unstable + public abstract void setUser(String user); + + @Public + @Unstable + public abstract String getQueue(); + + @Public + @Unstable + public abstract void setQueue(String queue); + + @Public + @Unstable + public abstract long getSubmitTime(); + + @Public + @Unstable + public abstract void setSubmitTime(long submitTime); + + @Public + @Unstable + public abstract long getStartTime(); + + @Public + @Unstable + public abstract void setStartTime(long startTime); + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ContainerFinishData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ContainerFinishData.java new file mode 100644 index 0000000..1c449e3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ContainerFinishData.java @@ -0,0 +1,99 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.util.Records; + +/** + * The class contains the fields that can be determined when + * RMContainer finishes, and that need to be stored persistently. + */ +@Public +@Unstable +public abstract class ContainerFinishData { + + @Public + @Unstable + public static ContainerFinishData newInstance(ContainerId containerId, + long finishTime, String diagnosticsInfo, String logURL, + int containerExitCode, ContainerState containerState) { + ContainerFinishData containerFD = + Records.newRecord(ContainerFinishData.class); + containerFD.setContainerId(containerId); + containerFD.setFinishTime(finishTime); + containerFD.setDiagnosticsInfo(diagnosticsInfo); + containerFD.setLogURL(logURL); + containerFD.setContainerExitStatus(containerExitCode); + containerFD.setContainerState(containerState); + return containerFD; + } + + @Public + @Unstable + public abstract ContainerId getContainerId(); + + @Public + @Unstable + public abstract void setContainerId(ContainerId containerId); + + @Public + @Unstable + public abstract long getFinishTime(); + + @Public + @Unstable + public abstract void setFinishTime(long finishTime); + + @Public + @Unstable + public abstract String getDiagnosticsInfo(); + + @Public + @Unstable + public abstract void setDiagnosticsInfo(String diagnosticsInfo); + + @Public + @Unstable + public abstract String getLogURL(); + + @Public + @Unstable + public abstract void setLogURL(String logURL); + + @Public + @Unstable + public abstract int getContainerExitStatus(); + + @Public + @Unstable + public abstract void setContainerExitStatus(int containerExitStatus); + + @Public + @Unstable + public abstract ContainerState getContainerState(); + + @Public + @Unstable + public abstract void setContainerState(ContainerState containerState); + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ContainerHistoryData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ContainerHistoryData.java new file mode 100644 index 0000000..f7660b2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ContainerHistoryData.java @@ -0,0 +1,197 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; + +/** + * The class contains all the fields that are stored persistently for + * RMContainer. + */ +@Public +@Unstable +public class ContainerHistoryData { + + private ContainerId containerId; + + private Resource allocatedResource; + + private NodeId assignedNode; + + private Priority priority; + + private long startTime; + + private long finishTime; + + private String diagnosticsInfo; + + private String logURL; + + private int containerExitStatus; + + private ContainerState containerState; + + @Public + @Unstable + public static ContainerHistoryData newInstance(ContainerId containerId, + Resource allocatedResource, NodeId assignedNode, Priority priority, + long startTime, long finishTime, String diagnosticsInfo, String logURL, + int containerExitCode, ContainerState containerState) { + ContainerHistoryData containerHD = new ContainerHistoryData(); + containerHD.setContainerId(containerId); + containerHD.setAllocatedResource(allocatedResource); + containerHD.setAssignedNode(assignedNode); + containerHD.setPriority(priority); + containerHD.setStartTime(startTime); + containerHD.setFinishTime(finishTime); + containerHD.setDiagnosticsInfo(diagnosticsInfo); + containerHD.setLogURL(logURL); + containerHD.setContainerExitStatus(containerExitCode); + containerHD.setContainerState(containerState); + return containerHD; + } + + @Public + @Unstable + public ContainerId getContainerId() { + return containerId; + } + + @Public + @Unstable + public void setContainerId(ContainerId containerId) { + this.containerId = containerId; + } + + @Public + @Unstable + public Resource getAllocatedResource() { + return allocatedResource; + } + + @Public + @Unstable + public void setAllocatedResource(Resource resource) { + this.allocatedResource = resource; + } + + @Public + @Unstable + public NodeId getAssignedNode() { + return assignedNode; + } + + @Public + @Unstable + public void setAssignedNode(NodeId nodeId) { + this.assignedNode = nodeId; + } + + @Public + @Unstable + public Priority getPriority() { + return priority; + } + + @Public + @Unstable + public void setPriority(Priority priority) { + this.priority = priority; + } + + @Public + @Unstable + public long getStartTime() { + return startTime; + } + + @Public + @Unstable + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + @Public + @Unstable + public long getFinishTime() { + return finishTime; + } + + @Public + @Unstable + public void setFinishTime(long finishTime) { + this.finishTime = finishTime; + } + + @Public + @Unstable + public String getDiagnosticsInfo() { + return diagnosticsInfo; + } + + @Public + @Unstable + public void setDiagnosticsInfo(String diagnosticsInfo) { + this.diagnosticsInfo = diagnosticsInfo; + } + + @Public + @Unstable + public String getLogURL() { + return logURL; + } + + @Public + @Unstable + public void setLogURL(String logURL) { + this.logURL = logURL; + } + + @Public + @Unstable + public int getContainerExitStatus() { + return containerExitStatus; + } + + @Public + @Unstable + public void setContainerExitStatus(int containerExitStatus) { + this.containerExitStatus = containerExitStatus; + } + + @Public + @Unstable + public ContainerState getContainerState() { + return containerState; + } + + @Public + @Unstable + public void setContainerState(ContainerState containerState) { + this.containerState = containerState; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ContainerStartData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ContainerStartData.java new file mode 100644 index 0000000..0c6dd81 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/ContainerStartData.java @@ -0,0 +1,92 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.util.Records; + +/** + * The class contains the fields that can be determined when + * RMContainer starts, and that need to be stored persistently. + */ +@Public +@Unstable +public abstract class ContainerStartData { + + @Public + @Unstable + public static ContainerStartData newInstance(ContainerId containerId, + Resource allocatedResource, NodeId assignedNode, Priority priority, + long startTime) { + ContainerStartData containerSD = + Records.newRecord(ContainerStartData.class); + containerSD.setContainerId(containerId); + containerSD.setAllocatedResource(allocatedResource); + containerSD.setAssignedNode(assignedNode); + containerSD.setPriority(priority); + containerSD.setStartTime(startTime); + return containerSD; + } + + @Public + @Unstable + public abstract ContainerId getContainerId(); + + @Public + @Unstable + public abstract void setContainerId(ContainerId containerId); + + @Public + @Unstable + public abstract Resource getAllocatedResource(); + + @Public + @Unstable + public abstract void setAllocatedResource(Resource resource); + + @Public + @Unstable + public abstract NodeId getAssignedNode(); + + @Public + @Unstable + public abstract void setAssignedNode(NodeId nodeId); + + @Public + @Unstable + public abstract Priority getPriority(); + + @Public + @Unstable + public abstract void setPriority(Priority priority); + + @Public + @Unstable + public abstract long getStartTime(); + + @Public + @Unstable + public abstract void setStartTime(long startTime); + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationAttemptFinishDataPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationAttemptFinishDataPBImpl.java new file mode 100644 index 0000000..7ae767b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationAttemptFinishDataPBImpl.java @@ -0,0 +1,239 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records.impl.pb; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationAttemptFinishDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationAttemptFinishDataProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto; +import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData; + +import com.google.protobuf.TextFormat; + +public class ApplicationAttemptFinishDataPBImpl + extends ApplicationAttemptFinishData { + + ApplicationAttemptFinishDataProto proto = + ApplicationAttemptFinishDataProto.getDefaultInstance(); + ApplicationAttemptFinishDataProto.Builder builder = null; + boolean viaProto = false; + + public ApplicationAttemptFinishDataPBImpl() { + builder = ApplicationAttemptFinishDataProto.newBuilder(); + } + + public ApplicationAttemptFinishDataPBImpl( + ApplicationAttemptFinishDataProto proto) { + this.proto = proto; + viaProto = true; + } + + private ApplicationAttemptId applicationAttemptId; + + @Override + public ApplicationAttemptId getApplicationAttemptId() { + if (this.applicationAttemptId != null) { + return this.applicationAttemptId; + } + ApplicationAttemptFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationAttemptId()) { + return null; + } + this.applicationAttemptId = + convertFromProtoFormat(p.getApplicationAttemptId()); + return this.applicationAttemptId; + } + + @Override + public void + setApplicationAttemptId(ApplicationAttemptId applicationAttemptId) { + maybeInitBuilder(); + if (applicationAttemptId == null) { + builder.clearApplicationAttemptId(); + } + this.applicationAttemptId = applicationAttemptId; + } + + @Override + public String getTrackingURL() { + ApplicationAttemptFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasTrackingUrl()) { + return null; + } + return p.getTrackingUrl(); + } + + @Override + public void setTrackingURL(String trackingURL) { + maybeInitBuilder(); + if (trackingURL == null) { + builder.clearTrackingUrl(); + return; + } + builder.setTrackingUrl(trackingURL); + } + + @Override + public String getDiagnosticsInfo() { + ApplicationAttemptFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasDiagnosticsInfo()) { + return null; + } + return p.getDiagnosticsInfo(); + } + + @Override + public void setDiagnosticsInfo(String diagnosticsInfo) { + maybeInitBuilder(); + if (diagnosticsInfo == null) { + builder.clearDiagnosticsInfo(); + return; + } + builder.setDiagnosticsInfo(diagnosticsInfo); + } + + @Override + public FinalApplicationStatus getFinalApplicationStatus() { + ApplicationAttemptFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasFinalApplicationStatus()) { + return null; + } + return convertFromProtoFormat(p.getFinalApplicationStatus()); + } + + @Override + public void setFinalApplicationStatus( + FinalApplicationStatus finalApplicationStatus) { + maybeInitBuilder(); + if (finalApplicationStatus == null) { + builder.clearFinalApplicationStatus(); + return; + } + builder.setFinalApplicationStatus( + convertToProtoFormat(finalApplicationStatus)); + } + + @Override + public YarnApplicationAttemptState getYarnApplicationAttemptState() { + ApplicationAttemptFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasYarnApplicationAttemptState()) { + return null; + } + return convertFromProtoFormat(p.getYarnApplicationAttemptState()); + } + + @Override + public void setYarnApplicationAttemptState(YarnApplicationAttemptState state) { + maybeInitBuilder(); + if (state == null) { + builder.clearYarnApplicationAttemptState(); + return; + } + builder.setYarnApplicationAttemptState(convertToProtoFormat(state)); + } + + public ApplicationAttemptFinishDataProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.applicationAttemptId != null && !((ApplicationAttemptIdPBImpl) + this.applicationAttemptId).getProto().equals( + builder.getApplicationAttemptId())) { + builder.setApplicationAttemptId( + convertToProtoFormat(this.applicationAttemptId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ApplicationAttemptFinishDataProto.newBuilder(proto); + } + viaProto = false; + } + + private ApplicationAttemptIdPBImpl convertFromProtoFormat( + ApplicationAttemptIdProto applicationAttemptId) { + return new ApplicationAttemptIdPBImpl(applicationAttemptId); + } + + private ApplicationAttemptIdProto convertToProtoFormat( + ApplicationAttemptId applicationAttemptId) { + return ((ApplicationAttemptIdPBImpl) applicationAttemptId).getProto(); + } + + private FinalApplicationStatus convertFromProtoFormat( + FinalApplicationStatusProto finalApplicationStatus) { + return ProtoUtils.convertFromProtoFormat(finalApplicationStatus); + } + + private FinalApplicationStatusProto convertToProtoFormat( + FinalApplicationStatus finalApplicationStatus) { + return ProtoUtils.convertToProtoFormat(finalApplicationStatus); + } + + private YarnApplicationAttemptStateProto convertToProtoFormat( + YarnApplicationAttemptState state) { + return ProtoUtils.convertToProtoFormat(state); + } + + private YarnApplicationAttemptState convertFromProtoFormat( + YarnApplicationAttemptStateProto yarnApplicationAttemptState) { + return ProtoUtils.convertFromProtoFormat(yarnApplicationAttemptState); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationAttemptStartDataPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationAttemptStartDataPBImpl.java new file mode 100644 index 0000000..5dbbecc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationAttemptStartDataPBImpl.java @@ -0,0 +1,211 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records.impl.pb; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationAttemptStartDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationAttemptStartDataProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData; + +import com.google.protobuf.TextFormat; + + +public class ApplicationAttemptStartDataPBImpl + extends ApplicationAttemptStartData { + + ApplicationAttemptStartDataProto proto = + ApplicationAttemptStartDataProto.getDefaultInstance(); + ApplicationAttemptStartDataProto.Builder builder = null; + boolean viaProto = false; + + public ApplicationAttemptStartDataPBImpl() { + builder = ApplicationAttemptStartDataProto.newBuilder(); + } + + public ApplicationAttemptStartDataPBImpl( + ApplicationAttemptStartDataProto proto) { + this.proto = proto; + viaProto = true; + } + + private ApplicationAttemptId applicationAttemptId; + private ContainerId masterContainerId; + + @Override + public ApplicationAttemptId getApplicationAttemptId() { + if (this.applicationAttemptId != null) { + return this.applicationAttemptId; + } + ApplicationAttemptStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationAttemptId()) { + return null; + } + this.applicationAttemptId = + convertFromProtoFormat(p.getApplicationAttemptId()); + return this.applicationAttemptId; + } + + @Override + public void + setApplicationAttemptId(ApplicationAttemptId applicationAttemptId) { + maybeInitBuilder(); + if (applicationAttemptId == null) { + builder.clearApplicationAttemptId(); + } + this.applicationAttemptId = applicationAttemptId; + } + + @Override + public String getHost() { + ApplicationAttemptStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasHost()) { + return null; + } + return p.getHost(); + } + + @Override + public void setHost(String host) { + maybeInitBuilder(); + if (host == null) { + builder.clearHost(); + return; + } + builder.setHost(host); + } + + @Override + public int getRPCPort() { + ApplicationAttemptStartDataProtoOrBuilder p = viaProto ? proto : builder; + return p.getRpcPort(); + } + + @Override + public void setRPCPort(int rpcPort) { + maybeInitBuilder(); + builder.setRpcPort(rpcPort); + } + + @Override + public ContainerId getMasterContainerId() { + if (this.masterContainerId != null) { + return this.masterContainerId; + } + ApplicationAttemptStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationAttemptId()) { + return null; + } + this.masterContainerId = + convertFromProtoFormat(p.getMasterContainerId()); + return this.masterContainerId; + } + + @Override + public void setMasterContainerId(ContainerId masterContainerId) { + maybeInitBuilder(); + if (masterContainerId == null) { + builder.clearMasterContainerId(); + } + this.masterContainerId = masterContainerId; + } + + public ApplicationAttemptStartDataProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.applicationAttemptId != null && !((ApplicationAttemptIdPBImpl) + this.applicationAttemptId).getProto().equals( + builder.getApplicationAttemptId())) { + builder.setApplicationAttemptId( + convertToProtoFormat(this.applicationAttemptId)); + } + if (this.masterContainerId != null && !((ContainerIdPBImpl) + this.masterContainerId).getProto().equals( + builder.getMasterContainerId())) { + builder.setMasterContainerId( + convertToProtoFormat(this.masterContainerId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ApplicationAttemptStartDataProto.newBuilder(proto); + } + viaProto = false; + } + + private ApplicationAttemptIdPBImpl convertFromProtoFormat( + ApplicationAttemptIdProto applicationAttemptId) { + return new ApplicationAttemptIdPBImpl(applicationAttemptId); + } + + private ApplicationAttemptIdProto convertToProtoFormat( + ApplicationAttemptId applicationAttemptId) { + return ((ApplicationAttemptIdPBImpl) applicationAttemptId).getProto(); + } + + private ContainerIdPBImpl convertFromProtoFormat( + ContainerIdProto containerId) { + return new ContainerIdPBImpl(containerId); + } + + private ContainerIdProto convertToProtoFormat( + ContainerId masterContainerId) { + return ((ContainerIdPBImpl) masterContainerId).getProto(); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationFinishDataPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationFinishDataPBImpl.java new file mode 100644 index 0000000..a56f7e3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationFinishDataPBImpl.java @@ -0,0 +1,228 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records.impl.pb; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationFinishDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationFinishDataProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto; +import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationFinishData; + +import com.google.protobuf.TextFormat; + + +public class ApplicationFinishDataPBImpl + extends ApplicationFinishData { + + ApplicationFinishDataProto proto = + ApplicationFinishDataProto.getDefaultInstance(); + ApplicationFinishDataProto.Builder builder = null; + boolean viaProto = false; + + private ApplicationId applicationId; + + public ApplicationFinishDataPBImpl() { + builder = ApplicationFinishDataProto.newBuilder(); + } + + public ApplicationFinishDataPBImpl(ApplicationFinishDataProto proto) { + this.proto = proto; + viaProto = true; + } + + @Override + public ApplicationId getApplicationId() { + if (this.applicationId != null) { + return this.applicationId; + } + ApplicationFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationId()) { + return null; + } + this.applicationId = convertFromProtoFormat(p.getApplicationId()); + return this.applicationId; + } + + @Override + public void setApplicationId(ApplicationId applicationId) { + maybeInitBuilder(); + if (applicationId == null) { + builder.clearApplicationId(); + } + this.applicationId = applicationId; + } + + @Override + public long getFinishTime() { + ApplicationFinishDataProtoOrBuilder p = viaProto ? proto : builder; + return p.getFinishTime(); + } + + @Override + public void setFinishTime(long finishTime) { + maybeInitBuilder(); + builder.setFinishTime(finishTime); + } + + @Override + public String getDiagnosticsInfo() { + ApplicationFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasDiagnosticsInfo()) { + return null; + } + return p.getDiagnosticsInfo(); + } + + @Override + public void setDiagnosticsInfo(String diagnosticsInfo) { + maybeInitBuilder(); + if (diagnosticsInfo == null) { + builder.clearDiagnosticsInfo(); + return; + } + builder.setDiagnosticsInfo(diagnosticsInfo); + } + + @Override + public FinalApplicationStatus getFinalApplicationStatus() { + ApplicationFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasFinalApplicationStatus()) { + return null; + } + return convertFromProtoFormat(p.getFinalApplicationStatus()); + } + + @Override + public void setFinalApplicationStatus( + FinalApplicationStatus finalApplicationStatus) { + maybeInitBuilder(); + if (finalApplicationStatus == null) { + builder.clearFinalApplicationStatus(); + return; + } + builder.setFinalApplicationStatus( + convertToProtoFormat(finalApplicationStatus)); + } + + @Override + public YarnApplicationState getYarnApplicationState() { + ApplicationFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasYarnApplicationState()) { + return null; + } + return convertFromProtoFormat(p.getYarnApplicationState()); + } + + @Override + public void setYarnApplicationState(YarnApplicationState state) { + maybeInitBuilder(); + if (state == null) { + builder.clearYarnApplicationState(); + return; + } + builder.setYarnApplicationState(convertToProtoFormat(state)); + } + + public ApplicationFinishDataProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.applicationId != null && !((ApplicationIdPBImpl) + this.applicationId).getProto().equals(builder.getApplicationId())) { + builder.setApplicationId(convertToProtoFormat(this.applicationId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ApplicationFinishDataProto.newBuilder(proto); + } + viaProto = false; + } + + private ApplicationIdProto convertToProtoFormat( + ApplicationId applicationId) { + return ((ApplicationIdPBImpl) applicationId).getProto(); + } + + private ApplicationIdPBImpl convertFromProtoFormat( + ApplicationIdProto applicationId) { + return new ApplicationIdPBImpl(applicationId); + } + + private FinalApplicationStatus convertFromProtoFormat( + FinalApplicationStatusProto finalApplicationStatus) { + return ProtoUtils.convertFromProtoFormat(finalApplicationStatus); + } + + private FinalApplicationStatusProto convertToProtoFormat( + FinalApplicationStatus finalApplicationStatus) { + return ProtoUtils.convertToProtoFormat(finalApplicationStatus); + } + + private YarnApplicationStateProto convertToProtoFormat( + YarnApplicationState state) { + return ProtoUtils.convertToProtoFormat(state); + } + + private YarnApplicationState convertFromProtoFormat( + YarnApplicationStateProto yarnApplicationState) { + return ProtoUtils.convertFromProtoFormat(yarnApplicationState); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationStartDataPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationStartDataPBImpl.java new file mode 100644 index 0000000..6629380 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ApplicationStartDataPBImpl.java @@ -0,0 +1,231 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records.impl.pb; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationStartDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ApplicationStartDataProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationStartData; + +import com.google.protobuf.TextFormat; + + +public class ApplicationStartDataPBImpl + extends ApplicationStartData { + + ApplicationStartDataProto proto = + ApplicationStartDataProto.getDefaultInstance(); + ApplicationStartDataProto.Builder builder = null; + boolean viaProto = false; + + private ApplicationId applicationId; + + public ApplicationStartDataPBImpl() { + builder = ApplicationStartDataProto.newBuilder(); + } + + public ApplicationStartDataPBImpl(ApplicationStartDataProto proto) { + this.proto = proto; + viaProto = true; + } + + @Override + public ApplicationId getApplicationId() { + if (this.applicationId != null) { + return this.applicationId; + } + ApplicationStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationId()) { + return null; + } + this.applicationId = convertFromProtoFormat(p.getApplicationId()); + return this.applicationId; + } + + @Override + public void setApplicationId(ApplicationId applicationId) { + maybeInitBuilder(); + if (applicationId == null) { + builder.clearApplicationId(); + } + this.applicationId = applicationId; + } + + @Override + public String getApplicationName() { + ApplicationStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationName()) { + return null; + } + return p.getApplicationName(); + } + + @Override + public void setApplicationName(String applicationName) { + maybeInitBuilder(); + if (applicationName == null) { + builder.clearApplicationName(); + return; + } + builder.setApplicationName(applicationName); + } + + @Override + public String getApplicationType() { + ApplicationStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasApplicationType()) { + return null; + } + return p.getApplicationType(); + } + + @Override + public void setApplicationType(String applicationType) { + maybeInitBuilder(); + if (applicationType == null) { + builder.clearApplicationType(); + return; + } + builder.setApplicationType(applicationType); + } + + @Override + public String getUser() { + ApplicationStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasUser()) { + return null; + } + return p.getUser(); + } + + @Override + public void setUser(String user) { + maybeInitBuilder(); + if (user == null) { + builder.clearUser(); + return; + } + builder.setUser(user); + } + + @Override + public String getQueue() { + ApplicationStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasQueue()) { + return null; + } + return p.getQueue(); + } + + @Override + public void setQueue(String queue) { + maybeInitBuilder(); + if (queue == null) { + builder.clearQueue(); + return; + } + builder.setQueue(queue); + } + + @Override + public long getSubmitTime() { + ApplicationStartDataProtoOrBuilder p = viaProto ? proto : builder; + return p.getSubmitTime(); + } + + @Override + public void setSubmitTime(long submitTime) { + maybeInitBuilder(); + builder.setSubmitTime(submitTime); + } + + @Override + public long getStartTime() { + ApplicationStartDataProtoOrBuilder p = viaProto ? proto : builder; + return p.getStartTime(); + } + + @Override + public void setStartTime(long startTime) { + maybeInitBuilder(); + builder.setStartTime(startTime); + } + + public ApplicationStartDataProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.applicationId != null && !((ApplicationIdPBImpl) + this.applicationId).getProto().equals(builder.getApplicationId())) { + builder.setApplicationId(convertToProtoFormat(this.applicationId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ApplicationStartDataProto.newBuilder(proto); + } + viaProto = false; + } + + private ApplicationIdProto convertToProtoFormat( + ApplicationId applicationId) { + return ((ApplicationIdPBImpl) applicationId).getProto(); + } + + private ApplicationIdPBImpl convertFromProtoFormat( + ApplicationIdProto applicationId) { + return new ApplicationIdPBImpl(applicationId); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ContainerFinishDataPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ContainerFinishDataPBImpl.java new file mode 100644 index 0000000..12fef73 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ContainerFinishDataPBImpl.java @@ -0,0 +1,226 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records.impl.pb; + +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ContainerFinishDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ContainerFinishDataProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerFinishData; + +import com.google.protobuf.TextFormat; + + +public class ContainerFinishDataPBImpl + extends ContainerFinishData { + + ContainerFinishDataProto proto = + ContainerFinishDataProto.getDefaultInstance(); + ContainerFinishDataProto.Builder builder = null; + boolean viaProto = false; + + private ContainerId containerId; + + public ContainerFinishDataPBImpl() { + builder = ContainerFinishDataProto.newBuilder(); + } + + public ContainerFinishDataPBImpl(ContainerFinishDataProto proto) { + this.proto = proto; + viaProto = true; + } + + @Override + public ContainerId getContainerId() { + if (this.containerId != null) { + return this.containerId; + } + ContainerFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasContainerId()) { + return null; + } + this.containerId = convertFromProtoFormat(p.getContainerId()); + return this.containerId; + } + + @Override + public void setContainerId(ContainerId containerId) { + maybeInitBuilder(); + if (containerId == null) { + builder.clearContainerId(); + } + this.containerId = containerId; + } + + @Override + public long getFinishTime() { + ContainerFinishDataProtoOrBuilder p = viaProto ? proto : builder; + return p.getFinishTime(); + } + + @Override + public void setFinishTime(long finishTime) { + maybeInitBuilder(); + builder.setFinishTime(finishTime); + } + + @Override + public String getDiagnosticsInfo() { + ContainerFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasDiagnosticsInfo()) { + return null; + } + return p.getDiagnosticsInfo(); + } + + @Override + public void setDiagnosticsInfo(String diagnosticsInfo) { + maybeInitBuilder(); + if (diagnosticsInfo == null) { + builder.clearDiagnosticsInfo(); + return; + } + builder.setDiagnosticsInfo(diagnosticsInfo); + } + + @Override + public String getLogURL() { + ContainerFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasLogUrl()) { + return null; + } + return p.getLogUrl(); + } + + @Override + public void setLogURL(String logURL) { + maybeInitBuilder(); + if (logURL == null) { + builder.clearLogUrl(); + return; + } + builder.setLogUrl(logURL); + } + + @Override + public int getContainerExitStatus() { + ContainerFinishDataProtoOrBuilder p = viaProto ? proto : builder; + return p.getContainerExitStatus(); + } + + @Override + public ContainerState getContainerState() { + ContainerFinishDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasContainerState()) { + return null; + } + return convertFromProtoFormat(p.getContainerState()); + } + + @Override + public void setContainerState(ContainerState state) { + maybeInitBuilder(); + if (state == null) { + builder.clearContainerState(); + return; + } + builder.setContainerState(convertToProtoFormat(state)); + } + + @Override + public void setContainerExitStatus(int containerExitStatus) { + maybeInitBuilder(); + builder.setContainerExitStatus(containerExitStatus); + } + + public ContainerFinishDataProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.containerId != null && !((ContainerIdPBImpl) + this.containerId).getProto().equals(builder.getContainerId())) { + builder.setContainerId(convertToProtoFormat(this.containerId)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ContainerFinishDataProto.newBuilder(proto); + } + viaProto = false; + } + + private ContainerIdProto convertToProtoFormat( + ContainerId containerId) { + return ((ContainerIdPBImpl) containerId).getProto(); + } + + private ContainerIdPBImpl convertFromProtoFormat( + ContainerIdProto containerId) { + return new ContainerIdPBImpl(containerId); + } + + private ContainerStateProto convertToProtoFormat( + ContainerState state) { + return ProtoUtils.convertToProtoFormat(state); + } + + private ContainerState convertFromProtoFormat( + ContainerStateProto containerState) { + return ProtoUtils.convertFromProtoFormat(containerState); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ContainerStartDataPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ContainerStartDataPBImpl.java new file mode 100644 index 0000000..b0b0cb4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/records/impl/pb/ContainerStartDataPBImpl.java @@ -0,0 +1,258 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.records.impl.pb; + +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.PriorityPBImpl; +import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ContainerStartDataProto; +import org.apache.hadoop.yarn.proto.ApplicationHistoryServerProtos.ContainerStartDataProtoOrBuilder; +import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; +import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto; +import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerStartData; + +import com.google.protobuf.TextFormat; + + +public class ContainerStartDataPBImpl + extends ContainerStartData { + + ContainerStartDataProto proto = + ContainerStartDataProto.getDefaultInstance(); + ContainerStartDataProto.Builder builder = null; + boolean viaProto = false; + + private ContainerId containerId; + private Resource resource; + private NodeId nodeId; + private Priority priority; + + public ContainerStartDataPBImpl() { + builder = ContainerStartDataProto.newBuilder(); + } + + public ContainerStartDataPBImpl(ContainerStartDataProto proto) { + this.proto = proto; + viaProto = true; + } + + @Override + public ContainerId getContainerId() { + if (this.containerId != null) { + return this.containerId; + } + ContainerStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasContainerId()) { + return null; + } + this.containerId = convertFromProtoFormat(p.getContainerId()); + return this.containerId; + } + + @Override + public void setContainerId(ContainerId containerId) { + maybeInitBuilder(); + if (containerId == null) { + builder.clearContainerId(); + } + this.containerId = containerId; + } + + @Override + public Resource getAllocatedResource() { + if (this.resource != null) { + return this.resource; + } + ContainerStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasAllocatedResource()) { + return null; + } + this.resource = convertFromProtoFormat(p.getAllocatedResource()); + return this.resource; + } + + @Override + public void setAllocatedResource(Resource resource) { + maybeInitBuilder(); + if (resource == null) { + builder.clearAllocatedResource(); + } + this.resource = resource; + } + + @Override + public NodeId getAssignedNode() { + if (this.nodeId != null) { + return this.nodeId; + } + ContainerStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasAssignedNodeId()) { + return null; + } + this.nodeId = convertFromProtoFormat(p.getAssignedNodeId()); + return this.nodeId; + } + + @Override + public void setAssignedNode(NodeId nodeId) { + maybeInitBuilder(); + if (nodeId == null) { + builder.clearAssignedNodeId(); + } + this.nodeId = nodeId; + } + + @Override + public Priority getPriority() { + if (this.priority != null) { + return this.priority; + } + ContainerStartDataProtoOrBuilder p = viaProto ? proto : builder; + if (!p.hasPriority()) { + return null; + } + this.priority = convertFromProtoFormat(p.getPriority()); + return this.priority; + } + + @Override + public void setPriority(Priority priority) { + maybeInitBuilder(); + if (priority == null) { + builder.clearPriority(); + } + this.priority = priority; + } + + @Override + public long getStartTime() { + ContainerStartDataProtoOrBuilder p = viaProto ? proto : builder; + return p.getStartTime(); + } + + @Override + public void setStartTime(long startTime) { + maybeInitBuilder(); + builder.setStartTime(startTime); + } + + public ContainerStartDataProto getProto() { + mergeLocalToProto(); + proto = viaProto ? proto : builder.build(); + viaProto = true; + return proto; + } + + @Override + public int hashCode() { + return getProto().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other == null) + return false; + if (other.getClass().isAssignableFrom(this.getClass())) { + return this.getProto().equals(this.getClass().cast(other).getProto()); + } + return false; + } + + @Override + public String toString() { + return TextFormat.shortDebugString(getProto()); + } + + private void mergeLocalToBuilder() { + if (this.containerId != null && !((ContainerIdPBImpl) + this.containerId).getProto().equals(builder.getContainerId())) { + builder.setContainerId(convertToProtoFormat(this.containerId)); + } + if (this.resource != null && !((ResourcePBImpl) + this.resource).getProto().equals(builder.getAllocatedResource())) { + builder.setAllocatedResource(convertToProtoFormat(this.resource)); + } + if (this.nodeId != null && !((NodeIdPBImpl) + this.nodeId).getProto().equals(builder.getAssignedNodeId())) { + builder.setAssignedNodeId(convertToProtoFormat(this.nodeId)); + } + if (this.priority != null && !((PriorityPBImpl) + this.priority).getProto().equals(builder.getPriority())) { + builder.setPriority(convertToProtoFormat(this.priority)); + } + } + + private void mergeLocalToProto() { + if (viaProto) { + maybeInitBuilder(); + } + mergeLocalToBuilder(); + proto = builder.build(); + viaProto = true; + } + + private void maybeInitBuilder() { + if (viaProto || builder == null) { + builder = ContainerStartDataProto.newBuilder(proto); + } + viaProto = false; + } + + private ContainerIdProto convertToProtoFormat( + ContainerId containerId) { + return ((ContainerIdPBImpl) containerId).getProto(); + } + + private ContainerIdPBImpl convertFromProtoFormat( + ContainerIdProto containerId) { + return new ContainerIdPBImpl(containerId); + } + + private ResourceProto convertToProtoFormat(Resource resource) { + return ((ResourcePBImpl) resource).getProto(); + } + + private ResourcePBImpl convertFromProtoFormat(ResourceProto resource) { + return new ResourcePBImpl(resource); + } + + private NodeIdProto convertToProtoFormat(NodeId nodeId) { + return ((NodeIdPBImpl) nodeId).getProto(); + } + + private NodeIdPBImpl convertFromProtoFormat(NodeIdProto nodeId) { + return new NodeIdPBImpl(nodeId); + } + + private PriorityProto convertToProtoFormat(Priority priority) { + return ((PriorityPBImpl) priority).getProto(); + } + + private PriorityPBImpl convertFromProtoFormat(PriorityProto priority) { + return new PriorityPBImpl(priority); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSController.java new file mode 100644 index 0000000..7bbb391 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSController.java @@ -0,0 +1,56 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import org.apache.hadoop.yarn.webapp.Controller; + +import com.google.inject.Inject; + + +public class AHSController extends Controller { + + @Inject + AHSController(RequestContext ctx) { + super(ctx); + } + + @Override + public void index() { + setTitle("Application History"); + } + + public void app() { + render(AppPage.class); + } + + public void appattempt() { + render(AppAttemptPage.class); + } + + public void container() { + render(ContainerPage.class); + } + + /** + * Render the logs page. + */ + public void logs() { + render(AHSLogsPage.class); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSLogsPage.java new file mode 100644 index 0000000..8821bc0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSLogsPage.java @@ -0,0 +1,55 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.ENTITY_STRING; + +import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.log.AggregatedLogsBlock; + +public class AHSLogsPage extends AHSView { + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSView# + * preHead(org.apache.hadoop .yarn.webapp.hamlet.Hamlet.HTML) + */ + @Override + protected void preHead(Page.HTML<_> html) { + String logEntity = $(ENTITY_STRING); + if (logEntity == null || logEntity.isEmpty()) { + logEntity = $(CONTAINER_ID); + } + if (logEntity == null || logEntity.isEmpty()) { + logEntity = "UNKNOWN"; + } + commonPreHead(html); + } + + /** + * The content of this page is the AggregatedLogsBlock + * + * @return AggregatedLogsBlock.class + */ + @Override + protected Class content() { + return AggregatedLogsBlock.class; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java new file mode 100644 index 0000000..9e83a44 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java @@ -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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.sjoin; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_STATE; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit; + +import org.apache.hadoop.yarn.server.webapp.AppsBlock; +import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout; + +// Do NOT rename/refactor this to AHSView as it will wreak havoc +// on Mac OS HFS +public class AHSView extends TwoColumnLayout { + static final int MAX_DISPLAY_ROWS = 100; // direct table rendering + static final int MAX_FAST_ROWS = 1000; // inline js array + + @Override + protected void preHead(Page.HTML<_> html) { + commonPreHead(html); + set(DATATABLES_ID, "apps"); + set(initID(DATATABLES, "apps"), appsTableInit()); + setTableStyles(html, "apps", ".queue {width:6em}", ".ui {width:8em}"); + + // Set the correct title. + String reqState = $(APP_STATE); + reqState = (reqState == null || reqState.isEmpty() ? "All" : reqState); + setTitle(sjoin(reqState, "Applications")); + } + + protected void commonPreHead(Page.HTML<_> html) { + set(ACCORDION_ID, "nav"); + set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}"); + } + + @Override + protected Class nav() { + return NavBlock.class; + } + + @Override + protected Class content() { + return AppsBlock.class; + } + + private String appsTableInit() { + // id, user, name, queue, starttime, finishtime, state, status, progress, ui + return tableInit() + .append(", 'aaData': appsTableData") + .append(", bDeferRender: true") + .append(", bProcessing: true") + + .append("\n, aoColumnDefs: ") + .append(getAppsTableColumnDefs()) + + // Sort by id upon page load + .append(", aaSorting: [[0, 'desc']]}").toString(); + } + + protected String getAppsTableColumnDefs() { + StringBuilder sb = new StringBuilder(); + return sb + .append("[\n") + .append("{'sType':'numeric', 'aTargets': [0]") + .append(", 'mRender': parseHadoopID }") + + .append("\n, {'sType':'numeric', 'aTargets': [5, 6]") + .append(", 'mRender': renderHadoopDate }") + + .append("\n, {'sType':'numeric', bSearchable:false, 'aTargets': [9]") + .append(", 'mRender': parseHadoopProgress }]").toString(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java new file mode 100644 index 0000000..1ebbc5e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java @@ -0,0 +1,51 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.pajoin; + +import org.apache.hadoop.yarn.server.api.ApplicationContext; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.WebApp; +import org.apache.hadoop.yarn.webapp.YarnWebParams; + +public class AHSWebApp extends WebApp implements YarnWebParams { + + private final ApplicationHistoryManager applicationHistoryManager; + + public AHSWebApp(ApplicationHistoryManager applicationHistoryManager) { + this.applicationHistoryManager = applicationHistoryManager; + } + + @Override + public void setup() { + bind(JAXBContextResolver.class); + bind(AHSWebServices.class); + bind(GenericExceptionHandler.class); + bind(ApplicationContext.class).toInstance(applicationHistoryManager); + route("/", AHSController.class); + route(pajoin("/apps", APP_STATE), AHSController.class); + route(pajoin("/app", APPLICATION_ID), AHSController.class, "app"); + route(pajoin("/appattempt", APPLICATION_ATTEMPT_ID), AHSController.class, + "appattempt"); + route(pajoin("/container", CONTAINER_ID), AHSController.class, "container"); + route(pajoin("/logs", NM_NODENAME, CONTAINER_ID, ENTITY_STRING, APP_OWNER, + CONTAINER_LOG_TYPE), AHSController.class, "logs"); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java new file mode 100644 index 0000000..01cfe4c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java @@ -0,0 +1,176 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import java.util.Collections; +import java.util.Set; + +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; + +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.server.api.ApplicationContext; +import org.apache.hadoop.yarn.server.webapp.WebServices; +import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo; +import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo; +import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo; +import org.apache.hadoop.yarn.webapp.BadRequestException; + +import com.google.inject.Inject; +import com.google.inject.Singleton; + +@Singleton +@Path("/ws/v1/applicationhistory") +public class AHSWebServices extends WebServices { + + @Inject + public AHSWebServices(ApplicationContext appContext) { + super(appContext); + } + + @GET + @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML }) + public AppsInfo get( + @Context HttpServletRequest req, + @Context HttpServletResponse res) { + return getApps(req, res, null, Collections. emptySet(), null, null, + null, null, null, null, null, null, Collections. emptySet()); + } + + @GET + @Path("/apps") + @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML }) + @Override + public AppsInfo getApps( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @QueryParam("state") String stateQuery, + @QueryParam("states") Set statesQuery, + @QueryParam("finalStatus") String finalStatusQuery, + @QueryParam("user") String userQuery, + @QueryParam("queue") String queueQuery, + @QueryParam("limit") String count, + @QueryParam("startedTimeBegin") String startedBegin, + @QueryParam("startedTimeEnd") String startedEnd, + @QueryParam("finishedTimeBegin") String finishBegin, + @QueryParam("finishedTimeEnd") String finishEnd, + @QueryParam("applicationTypes") Set applicationTypes) { + init(res); + validateStates(stateQuery, statesQuery); + return super.getApps(req, res, stateQuery, statesQuery, finalStatusQuery, + userQuery, queueQuery, count, startedBegin, startedEnd, finishBegin, + finishEnd, applicationTypes); + } + + @GET + @Path("/apps/{appid}") + @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML }) + @Override + public AppInfo getApp( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("appid") String appId) { + init(res); + return super.getApp(req, res, appId); + } + + @GET + @Path("/apps/{appid}/appattempts") + @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML }) + @Override + public AppAttemptsInfo getAppAttempts( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("appid") String appId) { + init(res); + return super.getAppAttempts(req, res, appId); + } + + @GET + @Path("/apps/{appid}/appattempts/{appattemptid}") + @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML }) + @Override + public AppAttemptInfo getAppAttempt( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("appid") String appId, + @PathParam("appattemptid") String appAttemptId) { + init(res); + return super.getAppAttempt(req, res, appId, appAttemptId); + } + + @GET + @Path("/apps/{appid}/appattempts/{appattemptid}/containers") + @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML }) + @Override + public ContainersInfo getContainers( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("appid") String appId, + @PathParam("appattemptid") String appAttemptId) { + init(res); + return super.getContainers(req, res, appId, appAttemptId); + } + + @GET + @Path("/apps/{appid}/appattempts/{appattemptid}/containers/{containerid}") + @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML }) + @Override + public ContainerInfo getContainer( + @Context HttpServletRequest req, + @Context HttpServletResponse res, + @PathParam("appid") String appId, + @PathParam("appattemptid") String appAttemptId, + @PathParam("containerid") String containerId) { + init(res); + return super.getContainer(req, res, appId, appAttemptId, containerId); + } + + private static void validateStates( + String stateQuery, Set statesQuery) { + // stateQuery is deprecated. + if (stateQuery != null && !stateQuery.isEmpty()) { + statesQuery.add(stateQuery); + } + Set appStates = parseQueries(statesQuery, true); + for (String appState : appStates) { + switch (YarnApplicationState.valueOf(appState.toUpperCase())) { + case FINISHED: + case FAILED: + case KILLED: + continue; + default: + throw new BadRequestException( + "Invalid application-state " + appState + + " specified. It should be a final state"); + } + } + } + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java new file mode 100644 index 0000000..99f6c76 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java @@ -0,0 +1,72 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit; + +import org.apache.hadoop.yarn.server.webapp.AppAttemptBlock; +import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.YarnWebParams; + +public class AppAttemptPage extends AHSView { + + @Override + protected void preHead(Page.HTML<_> html) { + commonPreHead(html); + + String appAttemptId = $(YarnWebParams.APPLICATION_ATTEMPT_ID); + set(TITLE, appAttemptId.isEmpty() ? + "Bad request: missing application attempt ID" : join( + "Application Attempt ", $(YarnWebParams.APPLICATION_ATTEMPT_ID))); + + set(DATATABLES_ID, "containers"); + set(initID(DATATABLES, "containers"), containersTableInit()); + setTableStyles(html, "containers", ".queue {width:6em}", ".ui {width:8em}"); + } + + @Override + protected Class content() { + return AppAttemptBlock.class; + } + + private String containersTableInit() { + return tableInit() + .append(", 'aaData': containersTableData") + .append(", bDeferRender: true") + .append(", bProcessing: true") + + .append("\n, aoColumnDefs: ") + .append(getContainersTableColumnDefs()) + + // Sort by id upon page load + .append(", aaSorting: [[0, 'desc']]}").toString(); + } + + protected String getContainersTableColumnDefs() { + StringBuilder sb = new StringBuilder(); + return sb + .append("[\n") + .append("{'sType':'numeric', 'aTargets': [0]") + .append(", 'mRender': parseHadoopID }]").toString(); + } + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java new file mode 100644 index 0000000..cf1abf2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java @@ -0,0 +1,73 @@ +/** +* 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit; + +import org.apache.hadoop.yarn.server.webapp.AppBlock; +import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.YarnWebParams; + +public class AppPage extends AHSView { + + @Override protected void preHead(Page.HTML<_> html) { + commonPreHead(html); + + String appId = $(YarnWebParams.APPLICATION_ID); + set(TITLE, appId.isEmpty() ? + "Bad request: missing application ID" : join( + "Application ", $(YarnWebParams.APPLICATION_ID))); + + set(DATATABLES_ID, "attempts"); + set(initID(DATATABLES, "attempts"), attemptsTableInit()); + setTableStyles(html, "attempts", ".queue {width:6em}", ".ui {width:8em}"); + } + + @Override protected Class content() { + return AppBlock.class; + } + + private String attemptsTableInit() { + return tableInit() + .append(", 'aaData': attemptsTableData") + .append(", bDeferRender: true") + .append(", bProcessing: true") + + .append("\n, aoColumnDefs: ") + .append(getAttemptsTableColumnDefs()) + + // Sort by id upon page load + .append(", aaSorting: [[0, 'desc']]}").toString(); + } + + protected String getAttemptsTableColumnDefs() { + StringBuilder sb = new StringBuilder(); + return sb + .append("[\n") + .append("{'sType':'numeric', 'aTargets': [0]") + .append(", 'mRender': parseHadoopID }") + + .append("\n, {'sType':'numeric', 'aTargets': [1]") + .append(", 'mRender': renderHadoopDate }]").toString(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java new file mode 100644 index 0000000..ff360a3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java @@ -0,0 +1,41 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.join; + +import org.apache.hadoop.yarn.server.webapp.ContainerBlock; +import org.apache.hadoop.yarn.webapp.SubView; +import org.apache.hadoop.yarn.webapp.YarnWebParams; + +public class ContainerPage extends AHSView { + + @Override + protected void preHead(Page.HTML<_> html) { + commonPreHead(html); + + String containerId = $(YarnWebParams.CONTAINER_ID); + set(TITLE, containerId.isEmpty() ? + "Bad request: missing container ID" : join( + "Container ", $(YarnWebParams.CONTAINER_ID))); + } + + @Override + protected Class content() { + return ContainerBlock.class; + }} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/JAXBContextResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/JAXBContextResolver.java new file mode 100644 index 0000000..7725d6e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/JAXBContextResolver.java @@ -0,0 +1,63 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +import javax.ws.rs.ext.ContextResolver; +import javax.ws.rs.ext.Provider; +import javax.xml.bind.JAXBContext; + +import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo; +import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo; +import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo; + +import com.google.inject.Singleton; +import com.sun.jersey.api.json.JSONConfiguration; +import com.sun.jersey.api.json.JSONJAXBContext; + +@Singleton +@Provider +@SuppressWarnings("rawtypes") +public class JAXBContextResolver implements ContextResolver { + + private JAXBContext context; + private final Set types; + + // you have to specify all the dao classes here + private final Class[] cTypes = { AppInfo.class, AppsInfo.class, + AppAttemptInfo.class, AppAttemptsInfo.class, ContainerInfo.class, + ContainersInfo.class }; + + public JAXBContextResolver() throws Exception { + this.types = new HashSet(Arrays.asList(cTypes)); + this.context = new JSONJAXBContext(JSONConfiguration.natural() + .rootUnwrapping(false).build(), cTypes); + } + + @Override + public JAXBContext getContext(Class objectType) { + return (types.contains(objectType)) ? context : null; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java new file mode 100644 index 0000000..e84ddec --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java @@ -0,0 +1,51 @@ +/** +* 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; + +public class NavBlock extends HtmlBlock { + + @Override + public void render(Block html) { + html. + div("#nav"). + h3("Application History"). + ul(). + li().a(url("apps"), "Applications"). + ul(). + li().a(url("apps", + YarnApplicationState.FINISHED.toString()), + YarnApplicationState.FINISHED.toString()). + _(). + li().a(url("apps", + YarnApplicationState.FAILED.toString()), + YarnApplicationState.FAILED.toString()). + _(). + li().a(url("apps", + YarnApplicationState.KILLED.toString()), + YarnApplicationState.KILLED.toString()). + _(). + _(). + _(). + _(). + _(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryStoreTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryStoreTestUtils.java new file mode 100644 index 0000000..f90ae09 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryStoreTestUtils.java @@ -0,0 +1,92 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerStartData; + +public class ApplicationHistoryStoreTestUtils { + + protected ApplicationHistoryStore store; + + protected void writeApplicationStartData(ApplicationId appId) + throws IOException { + store.applicationStarted( + ApplicationStartData.newInstance(appId, appId.toString(), "test type", + "test queue", + "test user", 0, 0)); + } + + protected void writeApplicationFinishData(ApplicationId appId) + throws IOException { + store.applicationFinished( + ApplicationFinishData.newInstance( + appId, 0, appId.toString(), FinalApplicationStatus.UNDEFINED, + YarnApplicationState.FINISHED)); + } + + protected void writeApplicationAttemptStartData( + ApplicationAttemptId appAttemptId) throws IOException { + store.applicationAttemptStarted( + ApplicationAttemptStartData.newInstance( + appAttemptId, appAttemptId.toString(), 0, + ContainerId.newInstance(appAttemptId, 1))); + } + + protected void writeApplicationAttemptFinishData( + ApplicationAttemptId appAttemptId) throws IOException { + store.applicationAttemptFinished( + ApplicationAttemptFinishData.newInstance(appAttemptId, + appAttemptId.toString(), "test tracking url", + FinalApplicationStatus.UNDEFINED, + YarnApplicationAttemptState.FINISHED)); + } + + protected void writeContainerStartData(ContainerId containerId) + throws IOException { + store.containerStarted( + ContainerStartData.newInstance(containerId, Resource.newInstance(0, 0), + NodeId.newInstance("localhost", 0), + Priority.newInstance(containerId.getId()), 0)); + } + + protected void writeContainerFinishData(ContainerId containerId) + throws IOException { + store.containerFinished( + ContainerFinishData.newInstance(containerId, 0, containerId.toString(), + "http://localhost:0/log", 0, ContainerState.COMPLETE)); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java new file mode 100644 index 0000000..f24fcbe --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java @@ -0,0 +1,189 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; +import java.util.List; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestApplicationHistoryClientService extends + ApplicationHistoryStoreTestUtils { + + ApplicationHistoryServer historyServer = null; + + @Before + public void setup() { + historyServer = new ApplicationHistoryServer(); + Configuration config = new YarnConfiguration(); + config.setClass(YarnConfiguration.AHS_STORE, + MemoryApplicationHistoryStore.class, ApplicationHistoryStore.class); + historyServer.init(config); + historyServer.start(); + store = ((ApplicationHistoryManagerImpl) historyServer + .getApplicationHistory()).getHistoryStore(); + } + + @After + public void tearDown() throws Exception { + historyServer.stop(); + } + + @Test + public void testApplicationReport() throws IOException, YarnException { + ApplicationId appId = null; + appId = ApplicationId.newInstance(0, 1); + writeApplicationStartData(appId); + writeApplicationFinishData(appId); + GetApplicationReportRequest request = GetApplicationReportRequest + .newInstance(appId); + GetApplicationReportResponse response = historyServer.getClientService() + .getClientHandler().getApplicationReport(request); + ApplicationReport appReport = response.getApplicationReport(); + Assert.assertNotNull(appReport); + Assert.assertEquals("application_0_0001", appReport.getApplicationId() + .toString()); + Assert.assertEquals("test type", appReport.getApplicationType().toString()); + Assert.assertEquals("test queue", appReport.getQueue().toString()); + } + + @Test + public void testApplications() throws IOException, YarnException { + ApplicationId appId = null; + appId = ApplicationId.newInstance(0, 1); + writeApplicationStartData(appId); + writeApplicationFinishData(appId); + ApplicationId appId1 = ApplicationId.newInstance(0, 2); + writeApplicationStartData(appId1); + writeApplicationFinishData(appId1); + GetApplicationsRequest request = GetApplicationsRequest.newInstance(); + GetApplicationsResponse response = historyServer.getClientService() + .getClientHandler().getApplications(request); + List appReport = response.getApplicationList(); + Assert.assertNotNull(appReport); + Assert.assertEquals(appId, appReport.get(0).getApplicationId()); + Assert.assertEquals(appId1, appReport.get(1).getApplicationId()); + } + + @Test + public void testApplicationAttemptReport() throws IOException, YarnException { + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, + 1); + writeApplicationAttemptStartData(appAttemptId); + writeApplicationAttemptFinishData(appAttemptId); + GetApplicationAttemptReportRequest request = GetApplicationAttemptReportRequest + .newInstance(appAttemptId); + GetApplicationAttemptReportResponse response = historyServer + .getClientService().getClientHandler().getApplicationAttemptReport( + request); + ApplicationAttemptReport attemptReport = response + .getApplicationAttemptReport(); + Assert.assertNotNull(attemptReport); + Assert.assertEquals("appattempt_0_0001_000001", attemptReport + .getApplicationAttemptId().toString()); + } + + @Test + public void testApplicationAttempts() throws IOException, YarnException { + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, + 1); + ApplicationAttemptId appAttemptId1 = ApplicationAttemptId.newInstance( + appId, 2); + writeApplicationAttemptStartData(appAttemptId); + writeApplicationAttemptFinishData(appAttemptId); + writeApplicationAttemptStartData(appAttemptId1); + writeApplicationAttemptFinishData(appAttemptId1); + GetApplicationAttemptsRequest request = GetApplicationAttemptsRequest + .newInstance(appId); + GetApplicationAttemptsResponse response = historyServer.getClientService() + .getClientHandler().getApplicationAttempts(request); + List attemptReports = response + .getApplicationAttemptList(); + Assert.assertNotNull(attemptReports); + Assert.assertEquals(appAttemptId, attemptReports.get(0) + .getApplicationAttemptId()); + Assert.assertEquals(appAttemptId1, attemptReports.get(1) + .getApplicationAttemptId()); + } + + @Test + public void testContainerReport() throws IOException, YarnException { + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, + 1); + ContainerId containerId = ContainerId.newInstance(appAttemptId, 1); + writeContainerStartData(containerId); + writeContainerFinishData(containerId); + GetContainerReportRequest request = GetContainerReportRequest + .newInstance(containerId); + GetContainerReportResponse response = historyServer.getClientService() + .getClientHandler().getContainerReport(request); + ContainerReport container = response.getContainerReport(); + Assert.assertNotNull(container); + Assert.assertEquals(containerId, container.getContainerId()); + } + + @Test + public void testContainers() throws IOException, YarnException { + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, + 1); + ContainerId containerId = ContainerId.newInstance(appAttemptId, 1); + ContainerId containerId1 = ContainerId.newInstance(appAttemptId, 2); + writeContainerStartData(containerId); + writeContainerFinishData(containerId); + writeContainerStartData(containerId1); + writeContainerFinishData(containerId1); + GetContainersRequest request = GetContainersRequest + .newInstance(appAttemptId); + GetContainersResponse response = historyServer.getClientService() + .getClientHandler().getContainers(request); + List containers = response.getContainerList(); + Assert.assertNotNull(containers); + Assert.assertEquals(containerId, containers.get(1).getContainerId()); + Assert.assertEquals(containerId1, containers.get(0).getContainerId()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerImpl.java new file mode 100644 index 0000000..15791f3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerImpl.java @@ -0,0 +1,74 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestApplicationHistoryManagerImpl extends + ApplicationHistoryStoreTestUtils { + ApplicationHistoryManagerImpl applicationHistoryManagerImpl = null; + + @Before + public void setup() throws Exception { + Configuration config = new Configuration(); + config.setClass(YarnConfiguration.AHS_STORE, + MemoryApplicationHistoryStore.class, ApplicationHistoryStore.class); + applicationHistoryManagerImpl = new ApplicationHistoryManagerImpl(); + applicationHistoryManagerImpl.init(config); + applicationHistoryManagerImpl.start(); + store = applicationHistoryManagerImpl.getHistoryStore(); + } + + @After + public void tearDown() throws Exception { + applicationHistoryManagerImpl.stop(); + } + + @Test + public void testApplicationReport() throws IOException, YarnException { + ApplicationId appId = null; + appId = ApplicationId.newInstance(0, 1); + writeApplicationStartData(appId); + writeApplicationFinishData(appId); + ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, + 1); + writeApplicationAttemptStartData(appAttemptId); + writeApplicationAttemptFinishData(appAttemptId); + ApplicationReport appReport = applicationHistoryManagerImpl + .getApplication(appId); + Assert.assertNotNull(appReport); + Assert.assertEquals(appId, appReport.getApplicationId()); + Assert.assertEquals(appAttemptId, appReport + .getCurrentApplicationAttemptId()); + Assert.assertEquals(appAttemptId.toString(), appReport.getHost()); + Assert.assertEquals("test type", appReport.getApplicationType().toString()); + Assert.assertEquals("test queue", appReport.getQueue().toString()); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java new file mode 100644 index 0000000..0ad48e2 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryServer.java @@ -0,0 +1,77 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.Service.STATE; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.junit.After; +import org.junit.Test; + +public class TestApplicationHistoryServer { + + ApplicationHistoryServer historyServer = null; + + // simple test init/start/stop ApplicationHistoryServer. Status should change. + @Test(timeout = 50000) + public void testStartStopServer() throws Exception { + historyServer = new ApplicationHistoryServer(); + Configuration config = new YarnConfiguration(); + historyServer.init(config); + assertEquals(STATE.INITED, historyServer.getServiceState()); + assertEquals(2, historyServer.getServices().size()); + ApplicationHistoryClientService historyService = historyServer + .getClientService(); + assertNotNull(historyServer.getClientService()); + assertEquals(STATE.INITED, historyService.getServiceState()); + + historyServer.start(); + assertEquals(STATE.STARTED, historyServer.getServiceState()); + assertEquals(STATE.STARTED, historyService.getServiceState()); + historyServer.stop(); + assertEquals(STATE.STOPPED, historyServer.getServiceState()); + } + + // test launch method + @Test(timeout = 60000) + public void testLaunch() throws Exception { + + ExitUtil.disableSystemExit(); + try { + historyServer = ApplicationHistoryServer + .launchAppHistoryServer(new String[0]); + } catch (ExitUtil.ExitException e) { + assertEquals(0, e.status); + ExitUtil.resetFirstExitException(); + fail(); + } + } + + @After + public void stop() { + if (historyServer != null) { + historyServer.stop(); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java new file mode 100644 index 0000000..d4a431f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java @@ -0,0 +1,198 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; +import java.net.URI; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerHistoryData; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestFileSystemApplicationHistoryStore extends + ApplicationHistoryStoreTestUtils { + + private FileSystem fs; + private Path fsWorkingPath; + + @Before + public void setup() throws Exception { + fs = new RawLocalFileSystem(); + Configuration conf = new Configuration(); + fs.initialize(new URI("/"), conf); + fsWorkingPath = new Path("Test"); + fs.delete(fsWorkingPath, true); + conf.set(YarnConfiguration.FS_HISTORY_STORE_URI, fsWorkingPath.toString()); + store = new FileSystemApplicationHistoryStore(); + store.init(conf); + store.start(); + } + + @After + public void tearDown() throws Exception { + store.stop(); + fs.delete(fsWorkingPath, true); + fs.close(); + } + + @Test + public void testReadWriteHistoryData() throws IOException { + testWriteHistoryData(5); + testReadHistoryData(5); + } + + private void testWriteHistoryData(int num) throws IOException { + // write application history data + for (int i = 1; i <= num; ++i) { + ApplicationId appId = ApplicationId.newInstance(0, i); + writeApplicationStartData(appId); + + // write application attempt history data + for (int j = 1; j <= num; ++j) { + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, j); + writeApplicationAttemptStartData(appAttemptId); + + // write container history data + for (int k = 1; k <= num; ++k) { + ContainerId containerId = ContainerId.newInstance(appAttemptId, k); + writeContainerStartData(containerId); + writeContainerFinishData(containerId); + + writeApplicationAttemptFinishData(appAttemptId); + } + } + + writeApplicationFinishData(appId); + } + } + + private void testReadHistoryData(int num) throws IOException { + // read application history data + Assert.assertEquals(num, store.getAllApplications().size()); + for (int i = 1; i <= num; ++i) { + ApplicationId appId = ApplicationId.newInstance(0, i); + ApplicationHistoryData appData = store.getApplication(appId); + Assert.assertNotNull(appData); + Assert.assertEquals(appId.toString(), appData.getApplicationName()); + Assert.assertEquals(appId.toString(), appData.getDiagnosticsInfo()); + + // read application attempt history data + Assert.assertEquals( + num, store.getApplicationAttempts(appId).size()); + for (int j = 1; j <= num; ++j) { + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, j); + ApplicationAttemptHistoryData attemptData = + store.getApplicationAttempt(appAttemptId); + Assert.assertNotNull(attemptData); + Assert.assertEquals(appAttemptId.toString(), attemptData.getHost()); + Assert.assertEquals(appAttemptId.toString(), + attemptData.getDiagnosticsInfo()); + + // read container history data + Assert.assertEquals( + num, store.getContainers(appAttemptId).size()); + for (int k = 1; k <= num; ++k) { + ContainerId containerId = ContainerId.newInstance(appAttemptId, k); + ContainerHistoryData containerData = store.getContainer(containerId); + Assert.assertNotNull(containerData); + Assert.assertEquals(Priority.newInstance(containerId.getId()), + containerData.getPriority()); + Assert.assertEquals(containerId.toString(), + containerData.getDiagnosticsInfo()); + } + ContainerHistoryData masterContainer = + store.getAMContainer(appAttemptId); + Assert.assertNotNull(masterContainer); + Assert.assertEquals(ContainerId.newInstance(appAttemptId, 1), + masterContainer.getContainerId()); + } + } + } + + @Test + public void testWriteAfterApplicationFinish() throws IOException { + ApplicationId appId = ApplicationId.newInstance(0, 1); + writeApplicationStartData(appId); + writeApplicationFinishData(appId); + // write application attempt history data + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + try { + writeApplicationAttemptStartData(appAttemptId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is not opened")); + } + try { + writeApplicationAttemptFinishData(appAttemptId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is not opened")); + } + // write container history data + ContainerId containerId = ContainerId.newInstance(appAttemptId, 1); + try { + writeContainerStartData(containerId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is not opened")); + } + try { + writeContainerFinishData(containerId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is not opened")); + } + } + + @Test + public void testMassiveWriteContainerHistoryData() throws IOException { + long mb = 1024 * 1024; + long usedDiskBefore = fs.getContentSummary(fsWorkingPath).getLength() / mb; + ApplicationId appId = ApplicationId.newInstance(0, 1); + writeApplicationStartData(appId); + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + for (int i = 1; i <= 100000; ++i) { + ContainerId containerId = ContainerId.newInstance(appAttemptId, i); + writeContainerStartData(containerId); + writeContainerFinishData(containerId); + } + writeApplicationFinishData(appId); + long usedDiskAfter = fs.getContentSummary(fsWorkingPath).getLength() / mb; + Assert.assertTrue((usedDiskAfter - usedDiskBefore) < 20); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestMemoryApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestMemoryApplicationHistoryStore.java new file mode 100644 index 0000000..51c01ff --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestMemoryApplicationHistoryStore.java @@ -0,0 +1,207 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice; + +import java.io.IOException; + +import junit.framework.Assert; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerHistoryData; +import org.junit.Before; +import org.junit.Test; + +public class TestMemoryApplicationHistoryStore extends + ApplicationHistoryStoreTestUtils { + + @Before + public void setup() { + store = new MemoryApplicationHistoryStore(); + } + + @Test + public void testReadWriteApplicationHistory() throws Exception { + // Out of order + ApplicationId appId = ApplicationId.newInstance(0, 1); + try { + writeApplicationFinishData(appId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains( + "is stored before the start information")); + } + // Normal + int numApps = 5; + for (int i = 1; i <= numApps; ++i) { + appId = ApplicationId.newInstance(0, i); + writeApplicationStartData(appId); + writeApplicationFinishData(appId); + } + Assert.assertEquals(numApps, store.getAllApplications().size()); + for (int i = 1; i <= numApps; ++i) { + appId = ApplicationId.newInstance(0, i); + ApplicationHistoryData data = store.getApplication(appId); + Assert.assertNotNull(data); + Assert.assertEquals(appId.toString(), data.getApplicationName()); + Assert.assertEquals(appId.toString(), data.getDiagnosticsInfo()); + } + // Write again + appId = ApplicationId.newInstance(0, 1); + try { + writeApplicationStartData(appId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is already stored")); + } + try { + writeApplicationFinishData(appId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is already stored")); + } + } + + @Test + public void testReadWriteApplicationAttemptHistory() throws Exception { + // Out of order + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + try { + writeApplicationAttemptFinishData(appAttemptId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains( + "is stored before the start information")); + } + // Normal + int numAppAttempts = 5; + writeApplicationStartData(appId); + for (int i = 1; i <= numAppAttempts; ++i) { + appAttemptId = + ApplicationAttemptId.newInstance(appId, i); + writeApplicationAttemptStartData(appAttemptId); + writeApplicationAttemptFinishData(appAttemptId); + } + Assert.assertEquals( + numAppAttempts, store.getApplicationAttempts(appId).size()); + for (int i = 1; i <= numAppAttempts; ++i) { + appAttemptId = + ApplicationAttemptId.newInstance(appId, i); + ApplicationAttemptHistoryData data = + store.getApplicationAttempt(appAttemptId); + Assert.assertNotNull(data); + Assert.assertEquals(appAttemptId.toString(), data.getHost()); + Assert.assertEquals(appAttemptId.toString(), data.getDiagnosticsInfo()); + } + writeApplicationFinishData(appId); + // Write again + appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + try { + writeApplicationAttemptStartData(appAttemptId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is already stored")); + } + try { + writeApplicationAttemptFinishData(appAttemptId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is already stored")); + } + } + + @Test + public void testReadWriteContainerHistory() throws Exception { + // Out of order + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + ContainerId containerId = ContainerId.newInstance(appAttemptId, 1); + try { + writeContainerFinishData(containerId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains( + "is stored before the start information")); + } + // Normal + writeApplicationAttemptStartData(appAttemptId); + int numContainers = 5; + for (int i = 1; i <= numContainers; ++i) { + containerId = ContainerId.newInstance(appAttemptId, i); + writeContainerStartData(containerId); + writeContainerFinishData(containerId); + } + Assert.assertEquals( + numContainers, store.getContainers(appAttemptId).size()); + for (int i = 1; i <= numContainers; ++i) { + containerId = ContainerId.newInstance(appAttemptId, i); + ContainerHistoryData data = store.getContainer(containerId); + Assert.assertNotNull(data); + Assert.assertEquals(Priority.newInstance(containerId.getId()), + data.getPriority()); + Assert.assertEquals(containerId.toString(), data.getDiagnosticsInfo()); + } + ContainerHistoryData masterContainer = store.getAMContainer(appAttemptId); + Assert.assertNotNull(masterContainer); + Assert.assertEquals(ContainerId.newInstance(appAttemptId, 1), + masterContainer.getContainerId()); + writeApplicationAttemptFinishData(appAttemptId); + // Write again + containerId = ContainerId.newInstance(appAttemptId, 1); + try { + writeContainerStartData(containerId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is already stored")); + } + try { + writeContainerFinishData(containerId); + Assert.fail(); + } catch (IOException e) { + Assert.assertTrue(e.getMessage().contains("is already stored")); + } + } + + @Test + public void testMassiveWriteContainerHistory() throws IOException { + long mb = 1024 * 1024; + Runtime runtime = Runtime.getRuntime(); + long usedMemoryBefore = (runtime.totalMemory() - runtime.freeMemory()) / mb; + int numContainers = 100000; + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + for (int i = 1; i <= numContainers; ++i) { + ContainerId containerId = ContainerId.newInstance(appAttemptId, i); + writeContainerStartData(containerId); + writeContainerFinishData(containerId); + } + long usedMemoryAfter = (runtime.totalMemory() - runtime.freeMemory()) / mb; + Assert.assertTrue((usedMemoryAfter - usedMemoryBefore) < 100); + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java new file mode 100644 index 0000000..472cc34 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java @@ -0,0 +1,184 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import static org.apache.hadoop.yarn.webapp.Params.TITLE; +import static org.mockito.Mockito.mock; +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.api.ApplicationContext; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManagerImpl; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStoreTestUtils; +import org.apache.hadoop.yarn.server.applicationhistoryservice.MemoryApplicationHistoryStore; +import org.apache.hadoop.yarn.util.StringHelper; +import org.apache.hadoop.yarn.webapp.YarnWebParams; +import org.apache.hadoop.yarn.webapp.test.WebAppTests; +import org.junit.Before; +import org.junit.Test; + +import com.google.inject.Injector; + +public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils { + + public void setApplicationHistoryStore(ApplicationHistoryStore store) { + this.store = store; + } + + @Before + public void setup() { + store = new MemoryApplicationHistoryStore(); + } + + @Test + public void testAppControllerIndex() throws Exception { + ApplicationHistoryManager ahManager = mock(ApplicationHistoryManager.class); + Injector injector = + WebAppTests.createMockInjector(ApplicationHistoryManager.class, + ahManager); + AHSController controller = + injector.getInstance(AHSController.class); + controller.index(); + Assert + .assertEquals("Application History", controller.get(TITLE, "unknown")); + } + + @Test + public void testView() throws Exception { + Injector injector = + WebAppTests.createMockInjector(ApplicationContext.class, + mockApplicationHistoryManager(5, 1, 1)); + AHSView ahsViewInstance = injector.getInstance(AHSView.class); + + ahsViewInstance.render(); + WebAppTests.flushOutput(injector); + + ahsViewInstance.set(YarnWebParams.APP_STATE, + YarnApplicationState.FAILED.toString()); + ahsViewInstance.render(); + WebAppTests.flushOutput(injector); + + ahsViewInstance.set(YarnWebParams.APP_STATE, StringHelper.cjoin( + YarnApplicationState.FAILED.toString(), YarnApplicationState.KILLED)); + ahsViewInstance.render(); + WebAppTests.flushOutput(injector); + } + + @Test + public void testAppPage() throws Exception { + Injector injector = + WebAppTests.createMockInjector(ApplicationContext.class, + mockApplicationHistoryManager(1, 5, 1)); + AppPage appPageInstance = injector.getInstance(AppPage.class); + + appPageInstance.render(); + WebAppTests.flushOutput(injector); + + appPageInstance.set(YarnWebParams.APPLICATION_ID, + ApplicationId.newInstance(0, 1).toString()); + appPageInstance.render(); + WebAppTests.flushOutput(injector); + } + + @Test + public void testAppAttemptPage() throws Exception { + Injector injector = + WebAppTests.createMockInjector(ApplicationContext.class, + mockApplicationHistoryManager(1, 1, 5)); + AppAttemptPage appAttemptPageInstance = + injector.getInstance(AppAttemptPage.class); + + appAttemptPageInstance.render(); + WebAppTests.flushOutput(injector); + + appAttemptPageInstance.set(YarnWebParams.APPLICATION_ATTEMPT_ID, + ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1) + .toString()); + appAttemptPageInstance.render(); + WebAppTests.flushOutput(injector); + } + + @Test + public void testContainerPage() throws Exception { + Injector injector = + WebAppTests.createMockInjector(ApplicationContext.class, + mockApplicationHistoryManager(1, 1, 1)); + ContainerPage containerPageInstance = + injector.getInstance(ContainerPage.class); + + containerPageInstance.render(); + WebAppTests.flushOutput(injector); + + containerPageInstance.set( + YarnWebParams.CONTAINER_ID, + ContainerId + .newInstance( + ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1), 1).toString()); + containerPageInstance.render(); + WebAppTests.flushOutput(injector); + } + + ApplicationHistoryManager mockApplicationHistoryManager( + int numApps, int numAppAttempts, int numContainers) throws Exception { + ApplicationHistoryManager ahManager = + new MockApplicationHistoryManagerImpl(store); + for (int i = 1; i <= numApps; ++i) { + ApplicationId appId = ApplicationId.newInstance(0, i); + writeApplicationStartData(appId); + for (int j = 1; j <= numAppAttempts; ++j) { + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, j); + writeApplicationAttemptStartData(appAttemptId); + for (int k = 1; k <= numContainers; ++k) { + ContainerId containerId = ContainerId.newInstance(appAttemptId, k); + writeContainerStartData(containerId); + writeContainerFinishData(containerId); + } + writeApplicationAttemptFinishData(appAttemptId); + } + writeApplicationFinishData(appId); + } + return ahManager; + } + + class MockApplicationHistoryManagerImpl extends + ApplicationHistoryManagerImpl { + + public MockApplicationHistoryManagerImpl(ApplicationHistoryStore store) { + super(); + init(new YarnConfiguration()); + start(); + } + + @Override + protected ApplicationHistoryStore createApplicationHistoryStore( + Configuration conf) { + return store; + } + }; + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java new file mode 100644 index 0000000..c5633fb --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java @@ -0,0 +1,296 @@ +/** + * 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.hadoop.yarn.server.applicationhistoryservice.webapp; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import javax.ws.rs.core.MediaType; + +import junit.framework.Assert; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.server.api.ApplicationContext; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore; +import org.apache.hadoop.yarn.server.applicationhistoryservice.MemoryApplicationHistoryStore; +import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; +import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import org.junit.Before; +import org.junit.Test; + +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.servlet.GuiceServletContextListener; +import com.google.inject.servlet.ServletModule; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.ClientResponse.Status; +import com.sun.jersey.api.client.UniformInterfaceException; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.test.framework.JerseyTest; +import com.sun.jersey.test.framework.WebAppDescriptor; + +public class TestAHSWebServices extends JerseyTest { + + private static ApplicationHistoryManager ahManager; + + private Injector injector = Guice.createInjector(new ServletModule() { + + @Override + protected void configureServlets() { + bind(JAXBContextResolver.class); + bind(AHSWebServices.class); + bind(GenericExceptionHandler.class); + try{ + ahManager = mockApplicationHistoryManager(); + } catch (Exception e) { + Assert.fail(); + } + bind(ApplicationContext.class).toInstance(ahManager); + serve("/*").with(GuiceContainer.class); + } + }); + + public class GuiceServletConfig extends GuiceServletContextListener { + + @Override + protected Injector getInjector() { + return injector; + } + } + + private ApplicationHistoryManager mockApplicationHistoryManager() + throws Exception { + ApplicationHistoryStore store = new MemoryApplicationHistoryStore(); + TestAHSWebApp testAHSWebApp = new TestAHSWebApp(); + testAHSWebApp.setApplicationHistoryStore(store); + ApplicationHistoryManager ahManager = + testAHSWebApp.mockApplicationHistoryManager(5, 5, 5); + return ahManager; + } + + public TestAHSWebServices() { + super(new WebAppDescriptor.Builder( + "org.apache.hadoop.yarn.server.applicationhistoryservice.webapp") + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class) + .contextPath("jersey-guice-filter").servletPath("/").build()); + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + } + + @Test + public void testInvalidUri() throws JSONException, Exception { + WebResource r = resource(); + String responseStr = ""; + try { + responseStr = r.path("ws").path("v1").path("applicationhistory") + .path("bogus").accept(MediaType.APPLICATION_JSON).get(String.class); + fail("should have thrown exception on invalid uri"); + } catch (UniformInterfaceException ue) { + ClientResponse response = ue.getResponse(); + assertEquals(Status.NOT_FOUND, response.getClientResponseStatus()); + + WebServicesTestUtils.checkStringMatch( + "error string exists and shouldn't", "", responseStr); + } + } + + @Test + public void testInvalidUri2() throws JSONException, Exception { + WebResource r = resource(); + String responseStr = ""; + try { + responseStr = r.accept(MediaType.APPLICATION_JSON).get(String.class); + fail("should have thrown exception on invalid uri"); + } catch (UniformInterfaceException ue) { + ClientResponse response = ue.getResponse(); + assertEquals(Status.NOT_FOUND, response.getClientResponseStatus()); + WebServicesTestUtils.checkStringMatch( + "error string exists and shouldn't", "", responseStr); + } + } + + @Test + public void testInvalidAccept() throws JSONException, Exception { + WebResource r = resource(); + String responseStr = ""; + try { + responseStr = r.path("ws").path("v1").path("applicationhistory") + .accept(MediaType.TEXT_PLAIN).get(String.class); + fail("should have thrown exception on invalid uri"); + } catch (UniformInterfaceException ue) { + ClientResponse response = ue.getResponse(); + assertEquals(Status.INTERNAL_SERVER_ERROR, + response.getClientResponseStatus()); + WebServicesTestUtils.checkStringMatch( + "error string exists and shouldn't", "", responseStr); + } + } + + @Test + public void testAppsQuery() throws Exception { + WebResource r = resource(); + ClientResponse response = r.path("ws").path("v1") + .path("applicationhistory").path("apps") + .queryParam("state", YarnApplicationState.FINISHED.toString()) + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + JSONObject json = response.getEntity(JSONObject.class); + assertEquals("incorrect number of elements", 1, json.length()); + JSONObject apps = json.getJSONObject("apps"); + assertEquals("incorrect number of elements", 1, apps.length()); + JSONArray array = apps.getJSONArray("app"); + assertEquals("incorrect number of elements", 5, array.length()); + } + + @Test + public void testSingleApp() throws Exception { + ApplicationId appId = ApplicationId.newInstance(0, 1); + WebResource r = resource(); + ClientResponse response = r.path("ws").path("v1") + .path("applicationhistory").path("apps").path(appId.toString()) + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + JSONObject json = response.getEntity(JSONObject.class); + assertEquals("incorrect number of elements", 1, json.length()); + JSONObject app = json.getJSONObject("app"); + assertEquals(appId.toString(), app.getString("appId")); + assertEquals(appId.toString(), app.get("name")); + assertEquals(appId.toString(), app.get("diagnosticsInfo")); + assertEquals("test queue", app.get("queue")); + assertEquals("test user", app.get("user")); + assertEquals("test type", app.get("type")); + assertEquals(FinalApplicationStatus.UNDEFINED.toString(), + app.get("finalAppStatus")); + assertEquals(YarnApplicationState.FINISHED.toString(), + app.get("appState")); + } + + @Test + public void testMultipleAttempts() throws Exception { + ApplicationId appId = ApplicationId.newInstance(0, 1); + WebResource r = resource(); + ClientResponse response = r.path("ws").path("v1") + .path("applicationhistory").path("apps").path(appId.toString()) + .path("appattempts").accept(MediaType.APPLICATION_JSON) + .get(ClientResponse.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + JSONObject json = response.getEntity(JSONObject.class); + assertEquals("incorrect number of elements", 1, json.length()); + JSONObject appAttempts = json.getJSONObject("appAttempts"); + assertEquals("incorrect number of elements", 1, appAttempts.length()); + JSONArray array = appAttempts.getJSONArray("appAttempt"); + assertEquals("incorrect number of elements", 5, array.length()); + } + + @Test + public void testSingleAttempt() throws Exception { + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + WebResource r = resource(); + ClientResponse response = r.path("ws").path("v1") + .path("applicationhistory").path("apps").path(appId.toString()) + .path("appattempts").path(appAttemptId.toString()) + .accept(MediaType.APPLICATION_JSON) + .get(ClientResponse.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + JSONObject json = response.getEntity(JSONObject.class); + assertEquals("incorrect number of elements", 1, json.length()); + JSONObject appAttempt = json.getJSONObject("appAttempt"); + assertEquals(appAttemptId.toString(), + appAttempt.getString("appAttemptId")); + assertEquals(appAttemptId.toString(), appAttempt.getString("host")); + assertEquals(appAttemptId.toString(), + appAttempt.getString("diagnosticsInfo")); + assertEquals("test tracking url", appAttempt.getString("trackingUrl")); + assertEquals(YarnApplicationAttemptState.FINISHED.toString(), + appAttempt.get("appAttemptState")); + } + + @Test + public void testMultipleContainers() throws Exception { + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + WebResource r = resource(); + ClientResponse response = r.path("ws").path("v1") + .path("applicationhistory").path("apps").path(appId.toString()) + .path("appattempts").path(appAttemptId.toString()) + .path("containers").accept(MediaType.APPLICATION_JSON) + .get(ClientResponse.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + JSONObject json = response.getEntity(JSONObject.class); + assertEquals("incorrect number of elements", 1, json.length()); + JSONObject containers = json.getJSONObject("containers"); + assertEquals("incorrect number of elements", 1, containers.length()); + JSONArray array = containers.getJSONArray("container"); + assertEquals("incorrect number of elements", 5, array.length()); + } + + @Test + public void testSingleContainer() throws Exception { + ApplicationId appId = ApplicationId.newInstance(0, 1); + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, 1); + ContainerId containerId = ContainerId.newInstance(appAttemptId, 1); + WebResource r = resource(); + ClientResponse response = r.path("ws").path("v1") + .path("applicationhistory").path("apps").path(appId.toString()) + .path("appattempts").path(appAttemptId.toString()) + .path("containers").path(containerId.toString()) + .accept(MediaType.APPLICATION_JSON) + .get(ClientResponse.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE, response.getType()); + JSONObject json = response.getEntity(JSONObject.class); + assertEquals("incorrect number of elements", 1, json.length()); + JSONObject container = json.getJSONObject("container"); + assertEquals(containerId.toString(), container.getString("containerId")); + assertEquals(containerId.toString(), + container.getString("diagnosticsInfo")); + assertEquals("0", container.getString("allocatedMB")); + assertEquals("0", container.getString("allocatedVCores")); + assertEquals(NodeId.newInstance("localhost", 0).toString(), + container.getString("assignedNodeId")); + assertEquals(Priority.newInstance(containerId.getId()).toString(), + container.getString("priority")); + assertEquals("http://localhost:0/log", container.getString("logUrl")); + assertEquals(ContainerState.COMPLETE.toString(), + container.getString("containerState")); + } + +} + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java new file mode 100644 index 0000000..6ac5d48 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java @@ -0,0 +1,121 @@ +/** + * 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.hadoop.yarn.server.api; + +import java.io.IOException; +import java.util.Map; + +import org.apache.hadoop.classification.InterfaceAudience.Public; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; + +@Public +@Unstable +public interface ApplicationContext { + /** + * This method returns Application {@link ApplicationReport} for the specified + * {@link ApplicationId}. + * + * @return {@link ApplicationReport} for the ApplicationId. + * @throws {@link IOException} + */ + @Public + @Unstable + ApplicationReport getApplication(ApplicationId appId) throws IOException; + + /** + * This method returns all Application {@link ApplicationReport}s + * + * @return map {@link ApplicationId, @link ApplicationReport}s. + * @throws {@link IOException} + */ + @Public + @Unstable + Map getAllApplications() throws IOException; + + /** + * Application can have multiple application attempts + * {@link ApplicationAttemptReport}. This method returns the all + * {@link ApplicationAttemptReport}s for the Application. + * + * @return all {@link ApplicationAttemptReport}s for the Application. + * @throws {@link IOException} + */ + @Public + @Unstable + Map getApplicationAttempts( + ApplicationId appId) throws IOException; + + /** + * This method returns {@link ApplicationAttemptReport} for specified + * {@link ApplicationId}. + * + * @param {@link ApplicationAttemptId} + * @return {@link ApplicationAttemptReport} for ApplicationAttemptId + * @throws {@link IOException} + */ + @Public + @Unstable + ApplicationAttemptReport getApplicationAttempt( + ApplicationAttemptId appAttemptId) throws IOException; + + /** + * This method returns {@link ContainerReport} for specified + * {@link ContainerId}. + * + * @param {@link ContainerId} + * @return {@link ContainerReport} for ContainerId + * @throws {@link IOException} + */ + @Public + @Unstable + ContainerReport getContainer(ContainerId containerId) throws IOException; + + /** + * This method returns {@link ContainerReport} for specified + * {@link ApplicationAttemptId}. + * + * @param {@link ApplicationAttemptId} + * @return {@link ContainerReport} for ApplicationAttemptId + * @throws {@link IOException} + */ + @Public + @Unstable + ContainerReport getAMContainer(ApplicationAttemptId appAttemptId) + throws IOException; + + /** + * This method returns Map{@link ContainerId,@link ContainerReport} for + * specified {@link ApplicationAttemptId}. + * + * @param {@link ApplicationAttemptId} + * @return Map{@link ContainerId, @link ContainerReport} for + * ApplicationAttemptId + * @throws {@link IOException} + */ + @Public + @Unstable + Map getContainers( + ApplicationAttemptId appAttemptId) throws IOException; +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java new file mode 100644 index 0000000..bdef62c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java @@ -0,0 +1,163 @@ +/** + * 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.hadoop.yarn.server.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.util.StringHelper.getPartUrl; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPLICATION_ATTEMPT_ID; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.server.api.ApplicationContext; +import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo; +import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; +import org.apache.hadoop.yarn.webapp.view.InfoBlock; + +import com.google.inject.Inject; + +public class AppAttemptBlock extends HtmlBlock { + + private static final Log LOG = LogFactory.getLog(AppAttemptBlock.class); + private final ApplicationContext appContext; + + @Inject + public AppAttemptBlock(ApplicationContext appContext) { + this.appContext = appContext; + } + + @Override + protected void render(Block html) { + String attemptid = $(APPLICATION_ATTEMPT_ID); + if (attemptid.isEmpty()) { + puts("Bad request: requires application attempt ID"); + return; + } + + ApplicationAttemptId appAttemptId = null; + try { + appAttemptId = ConverterUtils.toApplicationAttemptId(attemptid); + } catch (IllegalArgumentException e) { + puts("Invalid application attempt ID: " + attemptid); + return; + } + + ApplicationAttemptReport appAttemptReport; + try { + appAttemptReport = appContext.getApplicationAttempt(appAttemptId); + } catch (IOException e) { + String message = + "Failed to read the application attempt " + appAttemptId + "."; + LOG.error(message, e); + html.p()._(message)._(); + return; + } + if (appAttemptReport == null) { + puts("Application Attempt not found: " + attemptid); + return; + } + AppAttemptInfo appAttempt = new AppAttemptInfo(appAttemptReport); + + setTitle(join("Application Attempt ", attemptid)); + + String node = "N/A"; + if (appAttempt.getHost() != null && appAttempt.getRpcPort() >= 0 + && appAttempt.getRpcPort() < 65536) { + node = appAttempt.getHost() + ":" + appAttempt.getRpcPort(); + } + info("Application Attempt Overview"). + _("State", appAttempt.getAppAttemptState()). + _("Master Container", + appAttempt.getAmContainerId() == null ? "#" : root_url("container", + appAttempt.getAmContainerId()), + String.valueOf(appAttempt.getAmContainerId())). + _("Node:", node). + _("Tracking URL:", + appAttempt.getTrackingUrl() == null ? "#" : root_url(appAttempt + .getTrackingUrl()), "History"). + _("Diagnostics Info:", appAttempt.getDiagnosticsInfo()); + + html._(InfoBlock.class); + + Collection containers; + try { + containers = appContext.getContainers(appAttemptId).values(); + } catch (IOException e) { + html.p()._("Sorry, Failed to get containers for application attempt" + + attemptid + ".")._(); + return; + } + + // Container Table + TBODY> tbody = html. + table("#containers"). + thead(). + tr(). + th(".id", "Container ID"). + th(".node", "Node"). + th(".exitstatus", "Container Exit Status"). + th(".logs", "Logs")._()._(). + tbody(); + + StringBuilder containersTableData = new StringBuilder("[\n"); + for (ContainerReport containerReport : containers) { + String logURL = containerReport.getLogUrl(); + logURL = getPartUrl(logURL, "log"); + ContainerInfo container = new ContainerInfo(containerReport); + // ConatinerID numerical value parsed by parseHadoopID in yarn.dt.plugins.js + containersTableData + .append("[\"") + .append(container.getContainerId()) + .append("\",\"") + .append(StringEscapeUtils.escapeJavaScript( + StringEscapeUtils.escapeHtml(container.getAssignedNodeId()))) + .append("\",\"") + .append(container.getContainerExitStatus()) + .append("\",\"") + .append(logURL == null ? "N/A" : "Logs") + .append("\"],\n"); + } + if (containersTableData.charAt(containersTableData.length() - 2) == ',') { + containersTableData.delete(containersTableData.length() - 2, + containersTableData.length() - 1); + } + containersTableData.append("]"); + html.script().$type("text/javascript"). + _("var containersTableData=" + containersTableData)._(); + + + tbody._()._(); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java new file mode 100644 index 0000000..3d7877d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java @@ -0,0 +1,190 @@ +/** + * 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.hadoop.yarn.server.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.util.StringHelper.getPartUrl; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPLICATION_ID; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.hadoop.http.HttpConfig; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.server.api.ApplicationContext; +import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppInfo; +import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo; +import org.apache.hadoop.yarn.util.Apps; +import org.apache.hadoop.yarn.util.Times; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; +import org.apache.hadoop.yarn.webapp.view.InfoBlock; + +import com.google.inject.Inject; + +public class AppBlock extends HtmlBlock { + + protected ApplicationContext appContext; + + @Inject + AppBlock(ApplicationContext appContext, ViewContext ctx) { + super(ctx); + this.appContext = appContext; + } + + @Override + protected void render(Block html) { + String aid = $(APPLICATION_ID); + if (aid.isEmpty()) { + puts("Bad request: requires Application ID"); + return; + } + + ApplicationId appID = null; + try { + appID = Apps.toAppID(aid); + } catch (Exception e) { + puts("Invalid Application ID: " + aid); + return; + } + + ApplicationReport appReport; + try { + appReport = appContext.getApplication(appID); + } catch (IOException e) { + String message = + "Failed to read the application " + appID + "."; + LOG.error(message, e); + html.p()._(message)._(); + return; + } + if (appReport == null) { + puts("Application not found: " + aid); + return; + } + AppInfo app = new AppInfo(appReport); + + setTitle(join("Application ", aid)); + + info("Application Overview"). + _("User:", app.getUser()). + _("Name:", app.getName()). + _("Application Type:", app.getType()). + _("State:", app.getAppState()). + _("FinalStatus:", app.getFinalAppStatus()). + _("Started:", Times.format(app.getStartedTime())). + _("Elapsed:", StringUtils.formatTime( + Times.elapsed(app.getStartedTime(), app.getFinishedTime()))). + _("Tracking URL:", + app.getTrackingUrl() == null ? "#" : root_url(app.getTrackingUrl()), + "History"). + _("Diagnostics:", app.getDiagnosticsInfo()); + + html._(InfoBlock.class); + + Collection attempts; + try { + attempts = appContext.getApplicationAttempts(appID).values(); + } catch (IOException e) { + String message = + "Failed to read the attempts of the application " + appID + "."; + LOG.error(message, e); + html.p()._(message)._(); + return; + } + + // Application Attempt Table + TBODY> tbody = html. + table("#attempts"). + thead(). + tr(). + th(".id", "Attempt ID"). + th(".started", "Started"). + th(".node", "Node"). + th(".logs", "Logs")._()._(). + tbody(); + + StringBuilder attemptsTableData = new StringBuilder("[\n"); + for (ApplicationAttemptReport appAttemptReport : attempts) { + AppAttemptInfo appAttempt = new AppAttemptInfo(appAttemptReport); + ContainerReport containerReport; + try { + containerReport = appContext.getAMContainer(appAttemptReport + .getApplicationAttemptId()); + } catch (IOException e) { + String message = + "Failed to read the AM container of the application attempt " + + appAttemptReport.getApplicationAttemptId() + "."; + LOG.error(message, e); + html.p()._(message)._(); + return; + } + long startTime = Long.MAX_VALUE; + String logsLink = null; + if (containerReport != null) { + ContainerInfo container = new ContainerInfo(containerReport); + startTime = container.getStartedTime(); + logsLink = containerReport.getLogUrl(); + logsLink = getPartUrl(logsLink,"log"); + } + String nodeLink = null; + if (appAttempt.getHost() != null && appAttempt.getRpcPort() >= 0 + && appAttempt.getRpcPort() < 65536) { + nodeLink = appAttempt.getHost() + ":" + appAttempt.getRpcPort(); + } + // AppAttemptID numerical value parsed by parseHadoopID in yarn.dt.plugins.js + attemptsTableData + .append("[\"") + .append(appAttempt.getAppAttemptId()) + .append("\",\"") + .append(startTime) + .append("\",\"") + .append(nodeLink == null ? "N/A" : StringEscapeUtils.escapeJavaScript( + StringEscapeUtils.escapeHtml(nodeLink))) + .append("\",\"") + .append(nodeLink == null ? "N/A" : "Logs") + .append("\"],\n"); + } + if (attemptsTableData.charAt(attemptsTableData.length() - 2) == ',') { + attemptsTableData.delete(attemptsTableData.length() - 2, + attemptsTableData.length() - 1); + } + attemptsTableData.append("]"); + html.script().$type("text/javascript"). + _("var attemptsTableData=" + attemptsTableData)._(); + + + tbody._()._(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java new file mode 100644 index 0000000..7782863 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java @@ -0,0 +1,150 @@ +/** + * 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.hadoop.yarn.server.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_STATE; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR; +import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; + +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.server.api.ApplicationContext; +import org.apache.hadoop.yarn.server.webapp.dao.AppInfo; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE; +import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; + +import com.google.inject.Inject; + +public class AppsBlock extends HtmlBlock { + + protected ApplicationContext appContext; + + @Inject + AppsBlock(ApplicationContext appContext, ViewContext ctx) { + super(ctx); + this.appContext = appContext; + } + + @Override + public void render(Block html) { + setTitle("Applications"); + + TBODY> tbody = html. + table("#apps"). + thead(). + tr(). + th(".id", "ID"). + th(".user", "User"). + th(".name", "Name"). + th(".type", "Application Type"). + th(".queue", "Queue"). + th(".starttime", "StartTime"). + th(".finishtime", "FinishTime"). + th(".state", "State"). + th(".finalstatus", "FinalStatus"). + th(".progress", "Progress"). + th(".ui", "Tracking UI")._()._(). + tbody(); + Collection reqAppStates = null; + String reqStateString = $(APP_STATE); + if (reqStateString != null && !reqStateString.isEmpty()) { + String[] appStateStrings = reqStateString.split(","); + reqAppStates = new HashSet(appStateStrings.length); + for (String stateString : appStateStrings) { + reqAppStates.add(YarnApplicationState.valueOf(stateString)); + } + } + + Collection appReports; + try { + appReports = appContext.getAllApplications().values(); + } catch (IOException e) { + String message = + "Failed to read the applications."; + LOG.error(message, e); + html.p()._(message)._(); + return; + } + StringBuilder appsTableData = new StringBuilder("[\n"); + for (ApplicationReport appReport : appReports) { + if (reqAppStates != null + && !reqAppStates.contains(appReport.getYarnApplicationState())) { + continue; + } + AppInfo app = new AppInfo(appReport); + String percent = String.format("%.1f", app.getProgress()); + // AppID numerical value parsed by parseHadoopID in yarn.dt.plugins.js + appsTableData + .append("[\"") + .append(app.getAppId()) + .append("\",\"") + .append( + StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml( + app.getUser()))) + .append("\",\"") + .append( + StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml( + app.getName()))) + .append("\",\"") + .append( + StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml( + app.getType()))) + .append("\",\"") + .append( + StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml( + app.getQueue()))).append("\",\"") + .append(app.getStartedTime()).append("\",\"") + .append(app.getFinishedTime()).append("\",\"") + .append(app.getAppState()).append("\",\"") + .append(app.getFinalAppStatus()).append("\",\"") + // Progress bar + .append("
").append("
") + .append("\",\"") + .append("History").append("\"],\n"); + + } + if (appsTableData.charAt(appsTableData.length() - 2) == ',') { + appsTableData.delete(appsTableData.length() - 2, + appsTableData.length() - 1); + } + appsTableData.append("]"); + html.script().$type("text/javascript"). + _("var appsTableData=" + appsTableData)._(); + + tbody._()._(); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java new file mode 100644 index 0000000..c7abe1c --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java @@ -0,0 +1,102 @@ +/** + * 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.hadoop.yarn.server.webapp; + +import static org.apache.hadoop.yarn.util.StringHelper.join; +import static org.apache.hadoop.yarn.util.StringHelper.getPartUrl; +import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.server.api.ApplicationContext; +import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.Times; +import org.apache.hadoop.yarn.webapp.view.HtmlBlock; +import org.apache.hadoop.yarn.webapp.view.InfoBlock; + +import com.google.inject.Inject; + +public class ContainerBlock extends HtmlBlock { + + private static final Log LOG = LogFactory.getLog(ContainerBlock.class); + private final ApplicationContext appContext; + + @Inject + public ContainerBlock(ApplicationContext appContext, ViewContext ctx) { + super(ctx); + this.appContext = appContext; + } + + @Override + protected void render(Block html) { + String containerid = $(CONTAINER_ID); + if (containerid.isEmpty()) { + puts("Bad request: requires container ID"); + return; + } + + ContainerId containerId = null; + try { + containerId = ConverterUtils.toContainerId(containerid); + } catch (IllegalArgumentException e) { + puts("Invalid container ID: " + containerid); + return; + } + + ContainerReport containerReport; + try { + containerReport = appContext.getContainer(containerId); + } catch (IOException e) { + String message = "Failed to read the container " + containerid + "."; + LOG.error(message, e); + html.p()._(message)._(); + return; + } + if (containerReport == null) { + puts("Container not found: " + containerid); + return; + } + + ContainerInfo container = new ContainerInfo(containerReport); + String logURL = containerReport.getLogUrl(); + logURL = getPartUrl(logURL,"log"); + setTitle(join("Container ", containerid)); + + info("Container Overview"). + _("State:", container.getContainerState()). + _("Exit Status:", container.getContainerExitStatus()). + _("Node:", container.getAssignedNodeId()). + _("Priority:", container.getPriority()). + _("Started:", Times.format(container.getStartedTime())). + _("Elapsed:", StringUtils.formatTime( + Times.elapsed(container.getStartedTime(), + container.getFinishedTime()))). + _("Resource:", container.getAllocatedMB() + " Memory, " + + container.getAllocatedVCores() + " VCores"). + _("Logs:", logURL == null ? "#" : url(logURL), "Logs"). + _("Diagnostics:", container.getDiagnosticsInfo()); + + html._(InfoBlock.class); + } +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java new file mode 100644 index 0000000..d46b403 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java @@ -0,0 +1,378 @@ +/** + * 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.hadoop.yarn.server.webapp; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; + +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.WebApplicationException; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.server.api.ApplicationContext; +import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppInfo; +import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo; +import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo; +import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.webapp.BadRequestException; +import org.apache.hadoop.yarn.webapp.NotFoundException; + +public class WebServices { + + protected ApplicationContext appContext; + + public WebServices(ApplicationContext appContext) { + this.appContext = appContext; + } + + public AppsInfo getApps( + HttpServletRequest req, + HttpServletResponse res, + String stateQuery, + Set statesQuery, + String finalStatusQuery, + String userQuery, + String queueQuery, + String count, + String startedBegin, + String startedEnd, + String finishBegin, + String finishEnd, + Set applicationTypes) { + long num = 0; + boolean checkCount = false; + boolean checkStart = false; + boolean checkEnd = false; + boolean checkAppTypes = false; + boolean checkAppStates = false; + long countNum = 0; + + // set values suitable in case both of begin/end not specified + long sBegin = 0; + long sEnd = Long.MAX_VALUE; + long fBegin = 0; + long fEnd = Long.MAX_VALUE; + + if (count != null && !count.isEmpty()) { + checkCount = true; + countNum = Long.parseLong(count); + if (countNum <= 0) { + throw new BadRequestException("limit value must be greater then 0"); + } + } + + if (startedBegin != null && !startedBegin.isEmpty()) { + checkStart = true; + sBegin = Long.parseLong(startedBegin); + if (sBegin < 0) { + throw new BadRequestException("startedTimeBegin must be greater than 0"); + } + } + if (startedEnd != null && !startedEnd.isEmpty()) { + checkStart = true; + sEnd = Long.parseLong(startedEnd); + if (sEnd < 0) { + throw new BadRequestException("startedTimeEnd must be greater than 0"); + } + } + if (sBegin > sEnd) { + throw new BadRequestException( + "startedTimeEnd must be greater than startTimeBegin"); + } + + if (finishBegin != null && !finishBegin.isEmpty()) { + checkEnd = true; + fBegin = Long.parseLong(finishBegin); + if (fBegin < 0) { + throw new BadRequestException("finishTimeBegin must be greater than 0"); + } + } + if (finishEnd != null && !finishEnd.isEmpty()) { + checkEnd = true; + fEnd = Long.parseLong(finishEnd); + if (fEnd < 0) { + throw new BadRequestException("finishTimeEnd must be greater than 0"); + } + } + if (fBegin > fEnd) { + throw new BadRequestException( + "finishTimeEnd must be greater than finishTimeBegin"); + } + + Set appTypes = parseQueries(applicationTypes, false); + if (!appTypes.isEmpty()) { + checkAppTypes = true; + } + + // stateQuery is deprecated. + if (stateQuery != null && !stateQuery.isEmpty()) { + statesQuery.add(stateQuery); + } + Set appStates = parseQueries(statesQuery, true); + if (!appStates.isEmpty()) { + checkAppStates = true; + } + + AppsInfo allApps = new AppsInfo(); + Collection appReports = null; + try { + appReports = appContext.getAllApplications().values(); + } catch (IOException e) { + throw new WebApplicationException(e); + } + for (ApplicationReport appReport : appReports) { + + if (checkCount && num == countNum) { + break; + } + + if (checkAppStates && !appStates.contains( + appReport.getYarnApplicationState().toString().toLowerCase())) { + continue; + } + if (finalStatusQuery != null && !finalStatusQuery.isEmpty()) { + FinalApplicationStatus.valueOf(finalStatusQuery); + if (!appReport.getFinalApplicationStatus().toString() + .equalsIgnoreCase(finalStatusQuery)) { + continue; + } + } + if (userQuery != null && !userQuery.isEmpty()) { + if (!appReport.getUser().equals(userQuery)) { + continue; + } + } + if (queueQuery != null && !queueQuery.isEmpty()) { + if (!appReport.getQueue().equals(queueQuery)) { + continue; + } + } + if (checkAppTypes && !appTypes.contains( + appReport.getApplicationType().trim().toLowerCase())) { + continue; + } + + if (checkStart + && (appReport.getStartTime() < sBegin || + appReport.getStartTime() > sEnd)) { + continue; + } + if (checkEnd + && (appReport.getFinishTime() < fBegin || + appReport.getFinishTime() > fEnd)) { + continue; + } + AppInfo app = new AppInfo(appReport); + + allApps.add(app); + num++; + } + return allApps; + } + + public AppInfo getApp( + HttpServletRequest req, HttpServletResponse res, String appId) { + ApplicationId id = parseApplicationId(appId); + ApplicationReport app = null; + try { + app = appContext.getApplication(id); + } catch (IOException e) { + throw new WebApplicationException(e); + } + if (app == null) { + throw new NotFoundException("app with id: " + appId + " not found"); + } + return new AppInfo(app); + } + + public AppAttemptsInfo getAppAttempts( + HttpServletRequest req, HttpServletResponse res, String appId) { + ApplicationId id = parseApplicationId(appId); + Collection appAttemptReports = null; + try { + appAttemptReports = appContext.getApplicationAttempts(id).values(); + } catch (IOException e) { + throw new WebApplicationException(e); + } + AppAttemptsInfo appAttemptsInfo = new AppAttemptsInfo(); + for (ApplicationAttemptReport appAttemptReport : appAttemptReports) { + AppAttemptInfo appAttemptInfo = new AppAttemptInfo(appAttemptReport); + appAttemptsInfo.add(appAttemptInfo); + } + + return appAttemptsInfo; + } + + public AppAttemptInfo getAppAttempt( + HttpServletRequest req, HttpServletResponse res, + String appId, String appAttemptId) { + ApplicationId aid = parseApplicationId(appId); + ApplicationAttemptId aaid = parseApplicationAttemptId(appAttemptId); + validateIds(aid, aaid, null); + ApplicationAttemptReport appAttempt = null; + try { + appAttempt = appContext.getApplicationAttempt(aaid); + } catch (IOException e) { + throw new WebApplicationException(e); + } + if (appAttempt == null) { + throw new NotFoundException( + "app attempt with id: " + appAttemptId + " not found"); + } + return new AppAttemptInfo(appAttempt); + } + + public ContainersInfo getContainers( + HttpServletRequest req, HttpServletResponse res, + String appId, String appAttemptId) { + ApplicationId aid = parseApplicationId(appId); + ApplicationAttemptId aaid = parseApplicationAttemptId(appAttemptId); + validateIds(aid, aaid, null); + Collection containerReports = null; + try { + containerReports = appContext.getContainers(aaid).values(); + } catch (IOException e) { + throw new WebApplicationException(e); + } + ContainersInfo containersInfo = new ContainersInfo(); + for (ContainerReport containerReport : containerReports) { + ContainerInfo containerInfo = new ContainerInfo(containerReport); + containersInfo.add(containerInfo); + } + return containersInfo; + } + + public ContainerInfo getContainer( + HttpServletRequest req, HttpServletResponse res, + String appId, String appAttemptId, String containerId) { + ApplicationId aid = parseApplicationId(appId); + ApplicationAttemptId aaid = parseApplicationAttemptId(appAttemptId); + ContainerId cid = parseContainerId(containerId); + validateIds(aid, aaid, cid); + ContainerReport container = null; + try { + container = appContext.getContainer(cid); + } catch (IOException e) { + throw new WebApplicationException(e); + } + if (container == null) { + throw new NotFoundException( + "container with id: " + containerId + " not found"); + } + return new ContainerInfo(container); + } + protected void init(HttpServletResponse response) { + // clear content type + response.setContentType(null); + } + + protected static Set parseQueries( + Set queries, boolean isState) { + Set params = new HashSet(); + if (!queries.isEmpty()) { + for (String query : queries) { + if (query != null && !query.trim().isEmpty()) { + String[] paramStrs = query.split(","); + for (String paramStr : paramStrs) { + if (paramStr != null && !paramStr.trim().isEmpty()) { + if (isState) { + try { + // enum string is in the uppercase + YarnApplicationState.valueOf(paramStr.trim().toUpperCase()); + } catch (RuntimeException e) { + YarnApplicationState[] stateArray = + YarnApplicationState.values(); + String allAppStates = Arrays.toString(stateArray); + throw new BadRequestException( + "Invalid application-state " + paramStr.trim() + + " specified. It should be one of " + allAppStates); + } + } + params.add(paramStr.trim().toLowerCase()); + } + } + } + } + } + return params; + } + + protected static ApplicationId parseApplicationId(String appId) { + if (appId == null || appId.isEmpty()) { + throw new NotFoundException("appId, " + appId + ", is empty or null"); + } + ApplicationId aid = ConverterUtils.toApplicationId(appId); + if (aid == null) { + throw new NotFoundException("appId is null"); + } + return aid; + } + + protected static ApplicationAttemptId parseApplicationAttemptId( + String appAttemptId) { + if (appAttemptId == null || appAttemptId.isEmpty()) { + throw new NotFoundException( + "appAttemptId, " + appAttemptId + ", is empty or null"); + } + ApplicationAttemptId aaid = + ConverterUtils.toApplicationAttemptId(appAttemptId); + if (aaid == null) { + throw new NotFoundException("appAttemptId is null"); + } + return aaid; + } + + protected static ContainerId parseContainerId(String containerId) { + if (containerId == null || containerId.isEmpty()) { + throw new NotFoundException( + "containerId, " + containerId + ", is empty or null"); + } + ContainerId cid = ConverterUtils.toContainerId(containerId); + if (cid == null) { + throw new NotFoundException("containerId is null"); + } + return cid; + } + + protected void validateIds(ApplicationId appId, + ApplicationAttemptId appAttemptId, + ContainerId containerId) { + if (!appAttemptId.getApplicationId().equals(appId)) { + throw new NotFoundException("appId and appAttemptId don't match"); + } + if (containerId != null + && !containerId.getApplicationAttemptId().equals(appAttemptId)) { + throw new NotFoundException("appAttemptId and containerId don't match"); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java new file mode 100644 index 0000000..014ae8b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java @@ -0,0 +1,84 @@ +/** + * 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.hadoop.yarn.server.webapp.dao; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; + +@XmlRootElement(name = "appAttempt") +@XmlAccessorType(XmlAccessType.FIELD) +public class AppAttemptInfo { + + protected String appAttemptId; + protected String host; + protected int rpcPort; + protected String trackingUrl; + protected String diagnosticsInfo; + protected YarnApplicationAttemptState appAttemptState; + protected String amContainerId; + + public AppAttemptInfo() { + // JAXB needs this + } + + public AppAttemptInfo(ApplicationAttemptReport appAttempt) { + appAttemptId = appAttempt.getApplicationAttemptId().toString(); + host = appAttempt.getHost(); + rpcPort = appAttempt.getRpcPort(); + trackingUrl = appAttempt.getTrackingUrl(); + diagnosticsInfo = appAttempt.getDiagnostics(); + appAttemptState = appAttempt.getYarnApplicationAttemptState(); + if (appAttempt.getAMContainerId() != null) { + amContainerId = appAttempt.getAMContainerId().toString(); + } + } + + public String getAppAttemptId() { + return appAttemptId; + } + + public String getHost() { + return host; + } + + public int getRpcPort() { + return rpcPort; + } + + public String getTrackingUrl() { + return trackingUrl; + } + + public String getDiagnosticsInfo() { + return diagnosticsInfo; + } + + public YarnApplicationAttemptState getAppAttemptState() { + return appAttemptState; + } + + public String getAmContainerId() { + return amContainerId; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptsInfo.java new file mode 100644 index 0000000..23f0d8f --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptsInfo.java @@ -0,0 +1,47 @@ +/** + * 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 joblicable 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.hadoop.yarn.server.webapp.dao; + +import java.util.ArrayList; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlElement; +import javax.xml.bind.annotation.XmlRootElement; + +@XmlRootElement(name = "appAttempts") +@XmlAccessorType(XmlAccessType.FIELD) +public class AppAttemptsInfo { + + @XmlElement(name = "appAttempt") + protected ArrayList attempt = new ArrayList(); + + public AppAttemptsInfo() { + // JAXB needs this + } + + public void add(AppAttemptInfo info) { + this.attempt.add(info); + } + + public ArrayList getAttempts() { + return this.attempt; + } + +} + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java new file mode 100644 index 0000000..aedf0d3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java @@ -0,0 +1,169 @@ +/** + * 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.hadoop.yarn.server.webapp.dao; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.util.Times; + +@XmlRootElement(name = "app") +@XmlAccessorType(XmlAccessType.FIELD) +public class AppInfo { + + protected String appId; + protected String currentAppAttemptId; + protected String user; + protected String name; + protected String queue; + protected String type; + protected String host; + protected int rpcPort; + protected YarnApplicationState appState; + protected float progress; + protected String diagnosticsInfo; + protected String originalTrackingUrl; + protected String trackingUrl; + protected FinalApplicationStatus finalAppStatus; + protected long submittedTime; + protected long startedTime; + protected long finishedTime; + protected long elapsedTime; + protected int allocatedMB; + protected int allocatedVCores; + + public AppInfo() { + // JAXB needs this + } + + public AppInfo(ApplicationReport app) { + appId = app.getApplicationId().toString(); + if (app.getCurrentApplicationAttemptId() != null) { + currentAppAttemptId = app.getCurrentApplicationAttemptId().toString(); + } + user = app.getUser(); + queue = app.getQueue(); + name = app.getName(); + type = app.getApplicationType(); + host = app.getHost(); + rpcPort = app.getRpcPort(); + appState = app.getYarnApplicationState(); + diagnosticsInfo = app.getDiagnostics(); + trackingUrl = app.getTrackingUrl(); + originalTrackingUrl = app.getOriginalTrackingUrl(); + submittedTime = app.getStartTime(); + startedTime = app.getStartTime(); + finishedTime = app.getFinishTime(); + elapsedTime = Times.elapsed(startedTime, finishedTime); + finalAppStatus = app.getFinalApplicationStatus(); + ApplicationResourceUsageReport usage = + app.getApplicationResourceUsageReport(); + if (usage != null) { + allocatedMB = usage.getUsedResources().getMemory(); + allocatedVCores = usage.getUsedResources().getVirtualCores(); + } + progress = app.getProgress(); + } + + public String getAppId() { + return appId; + } + + public String getCurrentAppAttemptId() { + return currentAppAttemptId; + } + + public String getUser() { + return user; + } + + public String getName() { + return name; + } + + public String getQueue() { + return queue; + } + + public String getType() { + return type; + } + + public String getHost() { + return host; + } + + public int getRpcPort() { + return rpcPort; + } + + public YarnApplicationState getAppState() { + return appState; + } + + public float getProgress() { + return progress; + } + + public String getDiagnosticsInfo() { + return diagnosticsInfo; + } + + public String getOriginalTrackingUrl() { + return originalTrackingUrl; + } + + public String getTrackingUrl() { + return trackingUrl; + } + + public FinalApplicationStatus getFinalAppStatus() { + return finalAppStatus; + } + + public long getSubmittedTime() { + return submittedTime; + } + + public long getStartedTime() { + return startedTime; + } + + public long getFinishedTime() { + return finishedTime; + } + + public long getElapsedTime() { + return elapsedTime; + } + + public int getAllocatedMB() { + return allocatedMB; + } + + public int getAllocatedVCores() { + return allocatedVCores; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppsInfo.java new file mode 100644 index 0000000..f98b2ea --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppsInfo.java @@ -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.hadoop.yarn.server.webapp.dao; + +import java.util.ArrayList; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlRootElement; + +@XmlRootElement(name = "apps") +@XmlAccessorType(XmlAccessType.FIELD) +public class AppsInfo { + + protected ArrayList app = new ArrayList(); + + public AppsInfo() { + // JAXB needs this + } + + public void add(AppInfo appinfo) { + app.add(appinfo); + } + + public ArrayList getApps() { + return app; + } + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java new file mode 100644 index 0000000..2e4436e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java @@ -0,0 +1,117 @@ +/** + * 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.hadoop.yarn.server.webapp.dao; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.util.Times; + +@XmlRootElement(name = "container") +@XmlAccessorType(XmlAccessType.FIELD) +public class ContainerInfo { + + protected String containerId; + protected int allocatedMB; + protected int allocatedVCores; + protected String assignedNodeId; + protected int priority; + protected long startedTime; + protected long finishedTime; + protected long elapsedTime; + protected String diagnosticsInfo; + protected String logUrl; + protected int containerExitStatus; + protected ContainerState containerState; + + public ContainerInfo() { + // JAXB needs this + } + + public ContainerInfo(ContainerReport container) { + containerId = container.getContainerId().toString(); + if (container.getAllocatedResource() != null) { + allocatedMB = container.getAllocatedResource().getMemory(); + allocatedVCores = container.getAllocatedResource().getVirtualCores(); + } + if (container.getAssignedNode() != null) { + assignedNodeId = container.getAssignedNode().toString(); + } + priority = container.getPriority().getPriority(); + startedTime = container.getStartTime(); + finishedTime = container.getFinishTime(); + elapsedTime = Times.elapsed(startedTime, finishedTime); + diagnosticsInfo = container.getDiagnosticsInfo(); + logUrl = container.getLogUrl(); + containerExitStatus = container.getContainerExitStatus(); + containerState = container.getContainerState(); + } + + public String getContainerId() { + return containerId; + } + + public int getAllocatedMB() { + return allocatedMB; + } + + public int getAllocatedVCores() { + return allocatedVCores; + } + + public String getAssignedNodeId() { + return assignedNodeId; + } + + public int getPriority() { + return priority; + } + + public long getStartedTime() { + return startedTime; + } + + public long getFinishedTime() { + return finishedTime; + } + + public long getElapsedTime() { + return elapsedTime; + } + + public String getDiagnosticsInfo() { + return diagnosticsInfo; + } + + public String getLogUrl() { + return logUrl; + } + + public int getContainerExitStatus() { + return containerExitStatus; + } + + public ContainerState getContainerState() { + return containerState; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainersInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainersInfo.java new file mode 100644 index 0000000..49767e0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainersInfo.java @@ -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.hadoop.yarn.server.webapp.dao; + +import java.util.ArrayList; + +import javax.xml.bind.annotation.XmlAccessType; +import javax.xml.bind.annotation.XmlAccessorType; +import javax.xml.bind.annotation.XmlRootElement; + +@XmlRootElement(name = "containers") +@XmlAccessorType(XmlAccessType.FIELD) +public class ContainersInfo { + + protected ArrayList container = new ArrayList(); + + public ContainersInfo() { + // JAXB needs this + } + + public void add(ContainerInfo containerInfo) { + container.add(containerInfo); + } + + public ArrayList getContainers() { + return container; + } + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml index 1c13855..266872f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml @@ -39,6 +39,11 @@ org.apache.hadoop + hadoop-yarn-server-applicationhistoryservice + ${project.version} + + + org.apache.hadoop hadoop-yarn-server-web-proxy diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java index 7a8b2a2..29f0ebe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java @@ -421,26 +421,21 @@ public AllocateResponse allocate(AllocateRequest request) LOG.warn("Invalid blacklist request by application " + appAttemptId, e); throw e; } - - RMApp app = - this.rmContext.getRMApps().get(appAttemptId.getApplicationId()); - // In the case of work-preserving AM restart, it's possible for the - // AM to release containers from the earlier attempt. - if (!app.getApplicationSubmissionContext() - .getKeepContainersAcrossApplicationAttempts()) { - try { - RMServerUtils.validateContainerReleaseRequest(release, appAttemptId); - } catch (InvalidContainerReleaseException e) { - LOG.warn("Invalid container release by application " + appAttemptId, e); - throw e; - } + + try { + RMServerUtils.validateContainerReleaseRequest(release, appAttemptId); + } catch (InvalidContainerReleaseException e) { + LOG.warn("Invalid container release by application " + appAttemptId, e); + throw e; } - + // Send new requests to appAttempt. Allocation allocation = this.rScheduler.allocate(appAttemptId, ask, release, blacklistAdditions, blacklistRemovals); + RMApp app = this.rmContext.getRMApps().get( + appAttemptId.getApplicationId()); RMAppAttempt appAttempt = app.getRMAppAttempt(appAttemptId); AllocateResponse allocateResponse = @@ -596,4 +591,4 @@ public synchronized void setAllocateResponse(AllocateResponse response) { this.response = response; } } -} +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java index 1ddb1b4..64a4165 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java @@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; @@ -33,8 +34,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer; -import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM; +import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager; /** @@ -90,4 +91,10 @@ void setRMDelegationTokenSecretManager( RMDelegationTokenSecretManager delegationTokenSecretManager); + + RMApplicationHistoryWriter getRMApplicationHistoryWriter(); + + void setRMApplicationHistoryWriter( + RMApplicationHistoryWriter rmApplicationHistoryWriter); + } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java index ec90b4a..79e5983 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java @@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; @@ -76,6 +77,7 @@ private NodesListManager nodesListManager; private ResourceTrackerService resourceTrackerService; private ApplicationMasterService applicationMasterService; + private RMApplicationHistoryWriter rmApplicationHistoryWriter; /** * Default constructor. To be used in conjunction with setter methods for @@ -95,7 +97,8 @@ public RMContextImpl(Dispatcher rmDispatcher, AMRMTokenSecretManager appTokenSecretManager, RMContainerTokenSecretManager containerTokenSecretManager, NMTokenSecretManagerInRM nmTokenSecretManager, - ClientToAMTokenSecretManagerInRM clientToAMTokenSecretManager) { + ClientToAMTokenSecretManagerInRM clientToAMTokenSecretManager, + RMApplicationHistoryWriter rmApplicationHistoryWriter) { this(); this.setDispatcher(rmDispatcher); this.setContainerAllocationExpirer(containerAllocationExpirer); @@ -106,6 +109,7 @@ public RMContextImpl(Dispatcher rmDispatcher, this.setContainerTokenSecretManager(containerTokenSecretManager); this.setNMTokenSecretManager(nmTokenSecretManager); this.setClientToAMTokenSecretManager(clientToAMTokenSecretManager); + this.setRMApplicationHistoryWriter(rmApplicationHistoryWriter); RMStateStore nullStore = new NullRMStateStore(); nullStore.setRMDispatcher(rmDispatcher); @@ -318,4 +322,16 @@ public HAServiceState getHAServiceState() { return haServiceState; } } + + @Override + public RMApplicationHistoryWriter getRMApplicationHistoryWriter() { + return rmApplicationHistoryWriter; + } + + @Override + public void setRMApplicationHistoryWriter( + RMApplicationHistoryWriter rmApplicationHistoryWriter) { + this.rmApplicationHistoryWriter = rmApplicationHistoryWriter; + } + } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java index 8c73443..5556255 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java @@ -33,9 +33,14 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException; import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException; import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; @@ -43,6 +48,7 @@ * Utility methods to aid serving RM data through the REST and RPC APIs */ public class RMServerUtils { + public static List queryRMNodes(RMContext context, EnumSet acceptedStates) { // nodes contains nodes that are NEW, RUNNING OR UNHEALTHY @@ -56,7 +62,7 @@ } } } - + // inactiveNodes contains nodes that are DECOMMISSIONED, LOST, OR REBOOTED if (acceptedStates.contains(NodeState.DECOMMISSIONED) || acceptedStates.contains(NodeState.LOST) || @@ -69,7 +75,7 @@ } return results; } - + /** * Utility method to validate a list resource requests, by insuring that the * requested memory/vcore is non-negative and not greater than max @@ -85,8 +91,9 @@ public static void validateResourceRequests(List ask, * @throw InvalidResourceBlacklistRequestException if the * resource is not able to be added to the blacklist. */ - public static void validateBlacklistRequest(ResourceBlacklistRequest blacklistRequest) - throws InvalidResourceBlacklistRequestException { + public static void validateBlacklistRequest( + ResourceBlacklistRequest blacklistRequest) + throws InvalidResourceBlacklistRequestException { if (blacklistRequest != null) { List plus = blacklistRequest.getBlacklistAdditions(); if (plus != null && plus.contains(ResourceRequest.ANY)) { @@ -100,10 +107,12 @@ public static void validateBlacklistRequest(ResourceBlacklistRequest blacklistRe * It will validate to make sure all the containers belong to correct * application attempt id. If not then it will throw * {@link InvalidContainerReleaseException} - * @param containerReleaseList containers to be released as requested by - * application master. - * @param appAttemptId Application attempt Id - * @throws InvalidContainerReleaseException + * + * @param containerReleaseList + * containers to be released as requested by application master. + * @param appAttemptId + * Application attempt Id + * @throws InvalidContainerReleaseException */ public static void validateContainerReleaseRequest(List containerReleaseList, @@ -111,9 +120,11 @@ public static void validateBlacklistRequest(ResourceBlacklistRequest blacklistRe throws InvalidContainerReleaseException { for (ContainerId cId : containerReleaseList) { if (!appAttemptId.equals(cId.getApplicationAttemptId())) { - throw new InvalidContainerReleaseException("Cannot release container : " - + cId.toString() + " not belonging to this application attempt : " - + appAttemptId); + throw new InvalidContainerReleaseException( + "Cannot release container : " + + cId.toString() + + " not belonging to this application attempt : " + + appAttemptId); } } } @@ -157,4 +168,63 @@ public static UserGroupInformation verifyAccess( } return user; } + + public static YarnApplicationState createApplicationState( + RMAppState rmAppState) { + switch (rmAppState) { + case NEW: + return YarnApplicationState.NEW; + case NEW_SAVING: + return YarnApplicationState.NEW_SAVING; + case SUBMITTED: + return YarnApplicationState.SUBMITTED; + case ACCEPTED: + return YarnApplicationState.ACCEPTED; + case RUNNING: + return YarnApplicationState.RUNNING; + case FINISHING: + case FINISHED: + return YarnApplicationState.FINISHED; + case KILLED: + return YarnApplicationState.KILLED; + case FAILED: + return YarnApplicationState.FAILED; + default: + throw new YarnRuntimeException("Unknown state passed!"); + } + } + + public static YarnApplicationAttemptState createApplicationAttemptState( + RMAppAttemptState rmAppAttemptState) { + switch (rmAppAttemptState) { + case NEW: + return YarnApplicationAttemptState.NEW; + case SUBMITTED: + return YarnApplicationAttemptState.SUBMITTED; + case SCHEDULED: + return YarnApplicationAttemptState.SCHEDULED; + case ALLOCATED: + return YarnApplicationAttemptState.ALLOCATED; + case LAUNCHED: + return YarnApplicationAttemptState.LAUNCHED; + case ALLOCATED_SAVING: + case LAUNCHED_UNMANAGED_SAVING: + return YarnApplicationAttemptState.ALLOCATED_SAVING; + case RUNNING: + return YarnApplicationAttemptState.RUNNING; + case FINISHING: + return YarnApplicationAttemptState.FINISHING; + case FINAL_SAVING: + return YarnApplicationAttemptState.FINAL_SAVING; + case FINISHED: + return YarnApplicationAttemptState.FINISHED; + case KILLED: + return YarnApplicationAttemptState.KILLED; + case FAILED: + return YarnApplicationAttemptState.FAILED; + default: + throw new YarnRuntimeException("Unknown state passed!"); + } + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index 16c7ac7..0ee9104 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy; @@ -262,6 +263,10 @@ protected RMAppManager createRMAppManager() { this.applicationACLsManager, this.conf); } + protected RMApplicationHistoryWriter createRMApplicationHistoryWriter() { + return new RMApplicationHistoryWriter(); + } + // sanity check for configurations protected static void validateConfigs(Configuration conf) { // validate max-attempts @@ -346,6 +351,11 @@ protected void serviceInit(Configuration configuration) throws Exception { rmContext.setDelegationTokenRenewer(delegationTokenRenewer); } + RMApplicationHistoryWriter rmApplicationHistoryWriter = + createRMApplicationHistoryWriter(); + addService(rmApplicationHistoryWriter); + rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter); + // Register event handler for NodesListManager nodesListManager = new NodesListManager(rmContext); rmDispatcher.register(NodesListManagerEventType.class, nodesListManager); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java new file mode 100644 index 0000000..354fcc4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java @@ -0,0 +1,334 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.ahs; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.Event; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryWriter; +import org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationStartData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerFinishData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerStartData; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; + +/** + *

+ * {@link ResourceManager} uses this class to write the information of + * {@link RMApp}, {@link RMAppAttempt} and {@link RMContainer}. These APIs are + * non-blocking, and just schedule a writing history event. An self-contained + * dispatcher vector will handle the event in separate threads, and extract the + * required fields that are going to be persisted. Then, the extracted + * information will be persisted via the implementation of + * {@link ApplicationHistoryStore}. + *

+ */ +@Private +@Unstable +public class RMApplicationHistoryWriter extends CompositeService { + + public static final Log LOG = + LogFactory.getLog(RMApplicationHistoryWriter.class); + + private Dispatcher dispatcher; + private ApplicationHistoryWriter writer; + + public RMApplicationHistoryWriter() { + super(RMApplicationHistoryWriter.class.getName()); + } + + @Override + protected synchronized void serviceInit( + Configuration conf) throws Exception { + writer = createApplicationHistoryStore(conf); + addIfService(writer); + + dispatcher = createDispatcher(conf); + dispatcher.register( + WritingHistoryEventType.class, new ForwardingEventHandler()); + addIfService(dispatcher); + super.serviceInit(conf); + } + + protected Dispatcher createDispatcher(Configuration conf) { + MultiThreadedDispatcher dispatcher = new MultiThreadedDispatcher(conf.getInt( + YarnConfiguration.RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE, + YarnConfiguration.DEFAULT_RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE)); + dispatcher.setDrainEventsOnStop(); + return dispatcher; + } + + protected ApplicationHistoryStore createApplicationHistoryStore( + Configuration conf) { + boolean ahsEnabled = conf.getBoolean( + YarnConfiguration.RM_HISTORY_WRITER_ENABLED, + YarnConfiguration.DEFAULT_RM_HISTORY_WRITER_ENABLED); + // If the history writer is not enabled, a dummy store will be used to + // write nothing + if (ahsEnabled) { + try { + Class storeClass = + conf.getClass(YarnConfiguration.RM_HISTORY_WRITER_CLASS, + NullApplicationHistoryStore.class, + ApplicationHistoryStore.class); + return storeClass.newInstance(); + } catch (Exception e) { + String msg = "Could not instantiate ApplicationHistoryWriter: " + + conf.get(YarnConfiguration.RM_HISTORY_WRITER_CLASS, + NullApplicationHistoryStore.class.getName()); + LOG.error(msg, e); + throw new YarnRuntimeException(msg, e); + } + } else { + return new NullApplicationHistoryStore(); + } + } + + protected void handleWritingApplicationHistoryEvent( + WritingApplicationHistoryEvent event) { + switch (event.getType()) { + case APP_START: + WritingApplicationStartEvent wasEvent = + (WritingApplicationStartEvent) event; + try { + writer.applicationStarted(wasEvent.getApplicationStartData()); + LOG.info("Stored the start data of application " + + wasEvent.getApplicationId()); + } catch (IOException e) { + LOG.error("Error when storing the start data of application " + + wasEvent.getApplicationId()); + } + break; + case APP_FINISH: + WritingApplicationFinishEvent wafEvent = + (WritingApplicationFinishEvent) event; + try { + writer.applicationFinished(wafEvent.getApplicationFinishData()); + LOG.info("Stored the finish data of application " + + wafEvent.getApplicationId()); + } catch (IOException e) { + LOG.error("Error when storing the finish data of application " + + wafEvent.getApplicationId()); + } + break; + case APP_ATTEMPT_START: + WritingApplicationAttemptStartEvent waasEvent = + (WritingApplicationAttemptStartEvent) event; + try { + writer.applicationAttemptStarted(waasEvent + .getApplicationAttemptStartData()); + LOG.info("Stored the start data of application attempt " + + waasEvent.getApplicationAttemptId()); + } catch (IOException e) { + LOG.error("Error when storing the start data of application attempt " + + waasEvent.getApplicationAttemptId()); + } + break; + case APP_ATTEMPT_FINISH: + WritingApplicationAttemptFinishEvent waafEvent = + (WritingApplicationAttemptFinishEvent) event; + try { + writer.applicationAttemptFinished(waafEvent + .getApplicationAttemptFinishData()); + LOG.info("Stored the finish data of application attempt " + + waafEvent.getApplicationAttemptId()); + } catch (IOException e) { + LOG.error("Error when storing the finish data of application attempt " + + waafEvent.getApplicationAttemptId()); + } + break; + case CONTAINER_START: + WritingContainerStartEvent wcsEvent = + (WritingContainerStartEvent) event; + try { + writer.containerStarted(wcsEvent.getContainerStartData()); + LOG.info("Stored the start data of container " + + wcsEvent.getContainerId()); + } catch (IOException e) { + LOG.error("Error when storing the start data of container " + + wcsEvent.getContainerId()); + } + break; + case CONTAINER_FINISH: + WritingContainerFinishEvent wcfEvent = + (WritingContainerFinishEvent) event; + try { + writer.containerFinished(wcfEvent.getContainerFinishData()); + LOG.info("Stored the finish data of container " + + wcfEvent.getContainerId()); + } catch (IOException e) { + LOG.error("Error when storing the finish data of container " + + wcfEvent.getContainerId()); + } + break; + default: + LOG.error("Unknown WritingApplicationHistoryEvent type: " + + event.getType()); + } + } + + @SuppressWarnings("unchecked") + public void applicationStarted(RMApp app) { + dispatcher.getEventHandler().handle( + new WritingApplicationStartEvent(app.getApplicationId(), + ApplicationStartData.newInstance(app.getApplicationId(), + app.getName(), app.getApplicationType(), app.getQueue(), + app.getUser(), app.getSubmitTime(), app.getStartTime()))); + } + + @SuppressWarnings("unchecked") + public void applicationFinished(RMApp app) { + dispatcher.getEventHandler().handle( + new WritingApplicationFinishEvent(app.getApplicationId(), + ApplicationFinishData.newInstance(app.getApplicationId(), + app.getFinishTime(), + app.getDiagnostics().toString(), + app.getFinalApplicationStatus(), + app.createApplicationState()))); + } + + @SuppressWarnings("unchecked") + public void applicationAttemptStarted(RMAppAttempt appAttempt) { + dispatcher.getEventHandler().handle( + new WritingApplicationAttemptStartEvent(appAttempt.getAppAttemptId(), + ApplicationAttemptStartData.newInstance( + appAttempt.getAppAttemptId(), appAttempt.getHost(), + appAttempt.getRpcPort(), appAttempt.getMasterContainer() + .getId()))); + } + + @SuppressWarnings("unchecked") + public void applicationAttemptFinished(RMAppAttempt appAttempt) { + dispatcher.getEventHandler().handle( + new WritingApplicationAttemptFinishEvent(appAttempt.getAppAttemptId(), + ApplicationAttemptFinishData.newInstance(appAttempt + .getAppAttemptId(), appAttempt.getDiagnostics().toString(), + appAttempt.getTrackingUrl(), appAttempt + .getFinalApplicationStatus(), appAttempt + .createApplicationAttemptState()))); + } + + @SuppressWarnings("unchecked") + public void containerStarted(RMContainer container) { + dispatcher.getEventHandler().handle( + new WritingContainerStartEvent(container.getContainerId(), + ContainerStartData.newInstance(container.getContainerId(), + container.getAllocatedResource(), container.getAllocatedNode(), + container.getAllocatedPriority(), container.getStartTime()))); + } + + @SuppressWarnings("unchecked") + public void containerFinished(RMContainer container) { + dispatcher.getEventHandler().handle( + new WritingContainerFinishEvent(container.getContainerId(), + ContainerFinishData.newInstance(container.getContainerId(), + container.getFinishTime(), container.getDiagnosticsInfo(), + container.getLogURL(), container.getContainerExitStatus(), + container.getContainerState()))); + } + + /** + * EventHandler implementation which forward events to HistoryWriter Making + * use of it, HistoryWriter can avoid to have a public handle method + */ + private final class ForwardingEventHandler + implements EventHandler { + + @Override + public void handle(WritingApplicationHistoryEvent event) { + handleWritingApplicationHistoryEvent(event); + } + + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + protected static class MultiThreadedDispatcher + extends CompositeService implements Dispatcher { + + private List dispatchers = + new ArrayList(); + + public MultiThreadedDispatcher(int num) { + super(MultiThreadedDispatcher.class.getName()); + for (int i = 0; i < num; ++i) { + AsyncDispatcher dispatcher = createDispatcher(); + dispatchers.add(dispatcher); + addIfService(dispatcher); + } + } + + @Override + public EventHandler getEventHandler() { + return new CompositEventHandler(); + } + + @Override + public void register(Class eventType, EventHandler handler) { + for (AsyncDispatcher dispatcher : dispatchers) { + dispatcher.register(eventType, handler); + } + } + + public void setDrainEventsOnStop() { + for (AsyncDispatcher dispatcher : dispatchers) { + dispatcher.setDrainEventsOnStop(); + } + } + + private class CompositEventHandler implements EventHandler { + + @Override + public void handle(Event event) { + // Use hashCode (of ApplicationId) to dispatch the event to the child + // dispatcher, such that all the writing events of one application will + // be handled by one thread, the scheduled order of the these events + // will be preserved + int index = Math.abs(event.hashCode()) % dispatchers.size(); + dispatchers.get(index).getEventHandler().handle(event); + } + + } + + protected AsyncDispatcher createDispatcher() { + return new AsyncDispatcher(); + } + + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationAttemptFinishEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationAttemptFinishEvent.java new file mode 100644 index 0000000..3f6a620 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationAttemptFinishEvent.java @@ -0,0 +1,51 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.ahs; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData; + +public class WritingApplicationAttemptFinishEvent extends + WritingApplicationHistoryEvent { + + private ApplicationAttemptId appAttemptId; + private ApplicationAttemptFinishData appAttemptFinish; + + public WritingApplicationAttemptFinishEvent( + ApplicationAttemptId appAttemptId, + ApplicationAttemptFinishData appAttemptFinish) { + super(WritingHistoryEventType.APP_ATTEMPT_FINISH); + this.appAttemptId = appAttemptId; + this.appAttemptFinish = appAttemptFinish; + } + + @Override + public int hashCode() { + return appAttemptId.getApplicationId().hashCode(); + } + + public ApplicationAttemptId getApplicationAttemptId() { + return appAttemptId; + } + + public ApplicationAttemptFinishData getApplicationAttemptFinishData() { + return appAttemptFinish; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationAttemptStartEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationAttemptStartEvent.java new file mode 100644 index 0000000..7e092d3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationAttemptStartEvent.java @@ -0,0 +1,50 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.ahs; + +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData; + +public class WritingApplicationAttemptStartEvent extends + WritingApplicationHistoryEvent { + + private ApplicationAttemptId appAttemptId; + private ApplicationAttemptStartData appAttemptStart; + + public WritingApplicationAttemptStartEvent(ApplicationAttemptId appAttemptId, + ApplicationAttemptStartData appAttemptStart) { + super(WritingHistoryEventType.APP_ATTEMPT_START); + this.appAttemptId = appAttemptId; + this.appAttemptStart = appAttemptStart; + } + + @Override + public int hashCode() { + return appAttemptId.getApplicationId().hashCode(); + } + + public ApplicationAttemptId getApplicationAttemptId() { + return appAttemptId; + } + + public ApplicationAttemptStartData getApplicationAttemptStartData() { + return appAttemptStart; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationFinishEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationFinishEvent.java new file mode 100644 index 0000000..7a20214 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationFinishEvent.java @@ -0,0 +1,50 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.ahs; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationFinishData; + +public class WritingApplicationFinishEvent extends + WritingApplicationHistoryEvent { + + private ApplicationId appId; + private ApplicationFinishData appFinish; + + public WritingApplicationFinishEvent(ApplicationId appId, + ApplicationFinishData appFinish) { + super(WritingHistoryEventType.APP_FINISH); + this.appId = appId; + this.appFinish = appFinish; + } + + @Override + public int hashCode() { + return appId.hashCode(); + } + + public ApplicationId getApplicationId() { + return appId; + } + + public ApplicationFinishData getApplicationFinishData() { + return appFinish; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationHistoryEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationHistoryEvent.java new file mode 100644 index 0000000..bc17edc --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationHistoryEvent.java @@ -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.hadoop.yarn.server.resourcemanager.ahs; + +import org.apache.hadoop.yarn.event.AbstractEvent; + +public class WritingApplicationHistoryEvent extends + AbstractEvent { + + public WritingApplicationHistoryEvent(WritingHistoryEventType type) { + super(type); + } + +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationStartEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationStartEvent.java new file mode 100644 index 0000000..1b5dc78 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingApplicationStartEvent.java @@ -0,0 +1,50 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.ahs; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationStartData; + +public class WritingApplicationStartEvent extends + WritingApplicationHistoryEvent { + + private ApplicationId appId; + private ApplicationStartData appStart; + + public WritingApplicationStartEvent(ApplicationId appId, + ApplicationStartData appStart) { + super(WritingHistoryEventType.APP_START); + this.appId = appId; + this.appStart = appStart; + } + + @Override + public int hashCode() { + return appId.hashCode(); + } + + public ApplicationId getApplicationId() { + return appId; + } + + public ApplicationStartData getApplicationStartData() { + return appStart; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingContainerFinishEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingContainerFinishEvent.java new file mode 100644 index 0000000..6b27166 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingContainerFinishEvent.java @@ -0,0 +1,49 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.ahs; + +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerFinishData; + +public class WritingContainerFinishEvent extends WritingApplicationHistoryEvent { + + private ContainerId containerId; + private ContainerFinishData containerFinish; + + public WritingContainerFinishEvent(ContainerId containerId, + ContainerFinishData containerFinish) { + super(WritingHistoryEventType.CONTAINER_FINISH); + this.containerId = containerId; + this.containerFinish = containerFinish; + } + + @Override + public int hashCode() { + return containerId.getApplicationAttemptId().getApplicationId().hashCode(); + } + + public ContainerId getContainerId() { + return containerId; + } + + public ContainerFinishData getContainerFinishData() { + return containerFinish; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingContainerStartEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingContainerStartEvent.java new file mode 100644 index 0000000..f6df669 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingContainerStartEvent.java @@ -0,0 +1,49 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.ahs; + +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerStartData; + +public class WritingContainerStartEvent extends WritingApplicationHistoryEvent { + + private ContainerId containerId; + private ContainerStartData containerStart; + + public WritingContainerStartEvent(ContainerId containerId, + ContainerStartData containerStart) { + super(WritingHistoryEventType.CONTAINER_START); + this.containerId = containerId; + this.containerStart = containerStart; + } + + @Override + public int hashCode() { + return containerId.getApplicationAttemptId().getApplicationId().hashCode(); + } + + public ContainerId getContainerId() { + return containerId; + } + + public ContainerStartData getContainerStartData() { + return containerStart; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingHistoryEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingHistoryEventType.java new file mode 100644 index 0000000..cf27141 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/WritingHistoryEventType.java @@ -0,0 +1,29 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.ahs; + + +public enum WritingHistoryEventType { + APP_START, + APP_FINISH, + APP_ATTEMPT_START, + APP_ATTEMPT_FINISH, + CONTAINER_START, + CONTAINER_FINISH +} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppFailedAttemptEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppFailedAttemptEvent.java index f5c0f0f..111c6ac 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppFailedAttemptEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppFailedAttemptEvent.java @@ -23,20 +23,14 @@ public class RMAppFailedAttemptEvent extends RMAppEvent { private final String diagnostics; - private final boolean transferStateFromPreviousAttempt; public RMAppFailedAttemptEvent(ApplicationId appId, RMAppEventType event, - String diagnostics, boolean transferStateFromPreviousAttempt) { + String diagnostics) { super(appId, event); this.diagnostics = diagnostics; - this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt; } public String getDiagnostics() { return this.diagnostics; } - - public boolean getTransferStateFromPreviousAttempt() { - return transferStateFromPreviousAttempt; - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java index 2a90ec9..2b23ed5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java @@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent; import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState; import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable; @@ -63,7 +64,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppStartAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; @@ -77,7 +77,6 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.resource.Resources; -@SuppressWarnings({ "rawtypes", "unchecked" }) public class RMAppImpl implements RMApp, Recoverable { private static final Log LOG = LogFactory.getLog(RMAppImpl.class); @@ -335,6 +334,8 @@ public RMAppImpl(ApplicationId applicationId, RMContext rmContext, this.writeLock = lock.writeLock(); this.stateMachine = stateMachineFactory.make(this); + + rmContext.getRMApplicationHistoryWriter().applicationStarted(this); } @Override @@ -635,7 +636,7 @@ public void handle(RMAppEvent event) { this.writeLock.unlock(); } } - + @Override public void recover(RMState state) throws Exception{ ApplicationState appState = state.getApplicationState().get(getApplicationId()); @@ -648,28 +649,26 @@ public void recover(RMState state) throws Exception{ for(int i=0; i { @@ -732,6 +732,7 @@ public RMAppState transition(RMAppImpl app, RMAppEvent event) { private static final class AddApplicationToSchedulerTransition extends RMAppTransition { + @SuppressWarnings("unchecked") @Override public void transition(RMAppImpl app, RMAppEvent event) { if (event instanceof RMAppNewSavedEvent) { @@ -753,13 +754,14 @@ public void transition(RMAppImpl app, RMAppEvent event) { private static final class StartAppAttemptTransition extends RMAppTransition { @Override public void transition(RMAppImpl app, RMAppEvent event) { - app.createAndStartNewAttempt(false); + app.createNewAttempt(true); }; } private static final class FinalStateSavedTransition implements MultipleArcTransition { + @SuppressWarnings({ "rawtypes", "unchecked" }) @Override public RMAppState transition(RMAppImpl app, RMAppEvent event) { RMAppUpdateSavedEvent storeEvent = (RMAppUpdateSavedEvent) event; @@ -960,6 +962,7 @@ private static String getAppKilledDiagnostics() { } private static class KillAttemptTransition extends RMAppTransition { + @SuppressWarnings("unchecked") @Override public void transition(RMAppImpl app, RMAppEvent event) { app.stateBeforeKilling = app.getState(); @@ -987,6 +990,7 @@ public void transition(RMAppImpl app, RMAppEvent event) { return nodes; } + @SuppressWarnings("unchecked") public void transition(RMAppImpl app, RMAppEvent event) { Set nodes = getNodesOnWhichAttemptRan(app); for (NodeId nodeId : nodes) { @@ -1002,6 +1006,11 @@ public void transition(RMAppImpl app, RMAppEvent event) { app.handler.handle( new RMAppManagerEvent(app.applicationId, RMAppManagerEventType.APP_COMPLETED)); + + // TODO: We need to fix for the problem that RMApp enters the final state + // after RMAppAttempt in the killing case + app.rmContext.getRMApplicationHistoryWriter() + .applicationFinished(app); }; } @@ -1018,21 +1027,7 @@ public AttemptFailedTransition(RMAppState initialState) { public RMAppState transition(RMAppImpl app, RMAppEvent event) { if (!app.submissionContext.getUnmanagedAM() && app.attempts.size() < app.maxAppAttempts) { - boolean transferStateFromPreviousAttempt = false; - RMAppFailedAttemptEvent failedEvent = (RMAppFailedAttemptEvent) event; - transferStateFromPreviousAttempt = - failedEvent.getTransferStateFromPreviousAttempt(); - - RMAppAttempt oldAttempt = app.currentAttempt; - app.createAndStartNewAttempt(transferStateFromPreviousAttempt); - // Transfer the state from the previous attempt to the current attempt. - // Note that the previous failed attempt may still be collecting the - // container events from the scheduler and update its data structures - // before the new attempt is created. - if (transferStateFromPreviousAttempt) { - ((RMAppAttemptImpl) app.currentAttempt) - .transferStateFromPreviousAttempt(oldAttempt); - } + app.createNewAttempt(true); return initialState; } else { app.rememberTargetTransitionsAndStoreState(event, @@ -1069,27 +1064,7 @@ public YarnApplicationState createApplicationState() { if (rmAppState.equals(RMAppState.KILLING)) { rmAppState = stateBeforeKilling; } - switch (rmAppState) { - case NEW: - return YarnApplicationState.NEW; - case NEW_SAVING: - return YarnApplicationState.NEW_SAVING; - case SUBMITTED: - return YarnApplicationState.SUBMITTED; - case ACCEPTED: - return YarnApplicationState.ACCEPTED; - case RUNNING: - return YarnApplicationState.RUNNING; - case FINISHING: - case FINISHED: - return YarnApplicationState.FINISHED; - case KILLED: - return YarnApplicationState.KILLED; - case FAILED: - return YarnApplicationState.FAILED; - default: - throw new YarnRuntimeException("Unknown state passed!"); - } + return RMServerUtils.createApplicationState(rmAppState); } public static boolean isAppInFinalState(RMApp rmApp) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java index 335dbda..3a666dd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; @@ -178,4 +179,21 @@ * @return the start time of the application. */ long getStartTime(); + + /** + * The current state of the {@link RMAppAttempt}. + * + * @return the current state {@link RMAppAttemptState} for this application + * attempt. + */ + RMAppAttemptState getState(); + + /** + * Create the external user-facing state of the attempt of ApplicationMaster + * from the current state of the {@link RMAppAttempt}. + * + * @return the external user-facing state of the attempt ApplicationMaster. + */ + YarnApplicationAttemptState createApplicationAttemptState(); + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java index cd5c9d3..e07024d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java @@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; @@ -62,6 +63,7 @@ import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier; import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; @@ -129,9 +131,9 @@ private SecretKey clientTokenMasterKey = null; //nodes on while this attempt's containers ran - private Set ranNodes = + private final Set ranNodes = new HashSet(); - private List justFinishedContainers = + private final List justFinishedContainers = new ArrayList(); private Container masterContainer; @@ -148,7 +150,7 @@ private final StringBuilder diagnostics = new StringBuilder(); private Configuration conf; - private final boolean isLastAttempt; + private static final ExpiredTransition EXPIRED_TRANSITION = new ExpiredTransition(); @@ -330,12 +332,6 @@ RMAppAttemptEventType.KILL)) // Transitions from FAILED State - // For work-preserving AM restart, failed attempt are still capturing - // CONTAINER_FINISHED event and record the finished containers for the - // use by the next new attempt. - .addTransition(RMAppAttemptState.FAILED, RMAppAttemptState.FAILED, - RMAppAttemptEventType.CONTAINER_FINISHED, - new ContainerFinishedAtFailedTransition()) .addTransition( RMAppAttemptState.FAILED, RMAppAttemptState.FAILED, @@ -344,7 +340,8 @@ RMAppAttemptEventType.KILL, RMAppAttemptEventType.UNREGISTERED, RMAppAttemptEventType.STATUS_UPDATE, - RMAppAttemptEventType.CONTAINER_ALLOCATED)) + RMAppAttemptEventType.CONTAINER_ALLOCATED, + RMAppAttemptEventType.CONTAINER_FINISHED)) // Transitions from FINISHING State .addTransition(RMAppAttemptState.FINISHING, @@ -395,7 +392,7 @@ public RMAppAttemptImpl(ApplicationAttemptId appAttemptId, RMContext rmContext, YarnScheduler scheduler, ApplicationMasterService masterService, ApplicationSubmissionContext submissionContext, - Configuration conf, boolean isLastAttempt) { + Configuration conf) { this.conf = conf; this.applicationAttemptId = appAttemptId; this.rmContext = rmContext; @@ -409,7 +406,7 @@ public RMAppAttemptImpl(ApplicationAttemptId appAttemptId, this.writeLock = lock.writeLock(); this.proxiedTrackingUrl = generateProxyUriWithScheme(null); - this.isLastAttempt = isLastAttempt; + this.stateMachine = stateMachineFactory.make(this); } @@ -421,7 +418,7 @@ public ApplicationAttemptId getAppAttemptId() { @Override public ApplicationSubmissionContext getSubmissionContext() { return this.submissionContext; - } + } @Override public FinalApplicationStatus getFinalApplicationStatus() { @@ -690,11 +687,6 @@ public void recover(RMState state) throws Exception { this.startTime = attemptState.getStartTime(); } - public void transferStateFromPreviousAttempt(RMAppAttempt attempt) { - this.justFinishedContainers = attempt.getJustFinishedContainers(); - this.ranNodes = attempt.getRanNodes(); - } - private void recoverAppAttemptCredentials(Credentials appAttemptTokens) throws IOException { if (appAttemptTokens == null) { @@ -731,12 +723,6 @@ public void transition(RMAppAttemptImpl appAttempt, public void transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) { - boolean transferStateFromPreviousAttempt = false; - if (event instanceof RMAppStartAttemptEvent) { - transferStateFromPreviousAttempt = - ((RMAppStartAttemptEvent) event) - .getTransferStateFromPreviousAttempt(); - } appAttempt.startTime = System.currentTimeMillis(); // Register with the ApplicationMasterService @@ -756,10 +742,9 @@ public void transition(RMAppAttemptImpl appAttempt, new Token(id, appAttempt.rmContext.getAMRMTokenSecretManager()); - // Add the applicationAttempt to the scheduler and inform the scheduler - // whether to transfer the state from previous attempt. + // Add the applicationAttempt to the scheduler appAttempt.eventHandler.handle(new AppAttemptAddedSchedulerEvent( - appAttempt.applicationAttemptId, transferStateFromPreviousAttempt)); + appAttempt.applicationAttemptId)); } } @@ -998,7 +983,6 @@ public void transition(RMAppAttemptImpl appAttempt, // Tell the application and the scheduler ApplicationId applicationId = appAttemptId.getApplicationId(); RMAppEvent appEvent = null; - boolean keepContainersAcrossAppAttempts = false; switch (finalAttemptState) { case FINISHED: { @@ -1014,7 +998,7 @@ public void transition(RMAppAttemptImpl appAttempt, appEvent = new RMAppFailedAttemptEvent(applicationId, RMAppEventType.ATTEMPT_KILLED, - "Application killed by user.", false); + "Application killed by user."); } break; case FAILED: @@ -1022,17 +1006,10 @@ public void transition(RMAppAttemptImpl appAttempt, // don't leave the tracking URL pointing to a non-existent AM appAttempt.setTrackingUrlToRMAppPage(); appAttempt.invalidateAMHostAndPort(); - if (appAttempt.submissionContext - .getKeepContainersAcrossApplicationAttempts() - && !appAttempt.isLastAttempt - && !appAttempt.submissionContext.getUnmanagedAM()) { - keepContainersAcrossAppAttempts = true; - } appEvent = new RMAppFailedAttemptEvent(applicationId, - RMAppEventType.ATTEMPT_FAILED, appAttempt.getDiagnostics(), - keepContainersAcrossAppAttempts); - + RMAppEventType.ATTEMPT_FAILED, + appAttempt.getDiagnostics()); } break; default: @@ -1044,8 +1021,11 @@ public void transition(RMAppAttemptImpl appAttempt, appAttempt.eventHandler.handle(appEvent); appAttempt.eventHandler.handle(new AppAttemptRemovedSchedulerEvent( - appAttemptId, finalAttemptState, keepContainersAcrossAppAttempts)); + appAttemptId, finalAttemptState)); appAttempt.removeCredentials(appAttempt); + + appAttempt.rmContext.getRMApplicationHistoryWriter() + .applicationAttemptFinished(appAttempt); } } @@ -1070,11 +1050,6 @@ public void transition(RMAppAttemptImpl appAttempt, public void transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) { appAttempt.checkAttemptStoreError(event); - // TODO Today unmanaged AM client is waiting for app state to be Accepted to - // launch the AM. This is broken since we changed to start the attempt - // after the application is Accepted. We may need to introduce an attempt - // report that client can rely on to query the attempt state and choose to - // launch the unmanaged AM. super.transition(appAttempt, event); } } @@ -1143,6 +1118,9 @@ public void transition(RMAppAttemptImpl appAttempt, // write at AM launch time, so we don't save the AM's tracking URL anywhere // as that would mean an extra state-store write. For now, we hope that in // work-preserving restart, AMs are forced to reregister. + + appAttempt.rmContext.getRMApplicationHistoryWriter() + .applicationAttemptStarted(appAttempt); } } @@ -1376,20 +1354,6 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt, } } - private static final class ContainerFinishedAtFailedTransition - extends BaseTransition { - @Override - public void - transition(RMAppAttemptImpl appAttempt, RMAppAttemptEvent event) { - RMAppAttemptContainerFinishedEvent containerFinishedEvent = - (RMAppAttemptContainerFinishedEvent) event; - ContainerStatus containerStatus = - containerFinishedEvent.getContainerStatus(); - // Normal container. Add it in completed containers list - appAttempt.justFinishedContainers.add(containerStatus); - } - } - private static class ContainerFinishedFinalStateSavedTransition extends BaseTransition { @Override @@ -1514,6 +1478,23 @@ public long getStartTime() { } } + @Override + public RMAppAttemptState getState() { + this.readLock.lock(); + + try { + return this.stateMachine.getCurrentState(); + } finally { + this.readLock.unlock(); + } + } + + @Override + public YarnApplicationAttemptState createApplicationAttemptState() { + RMAppAttemptState state = getState(); + return RMServerUtils.createApplicationAttemptState(state); + } + private void launchAttempt(){ // Send event to launch the AM Container eventHandler.handle(new AMLauncherEvent(AMLauncherEventType.LAUNCH, this)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppStartAttemptEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppStartAttemptEvent.java deleted file mode 100644 index 55dab60..0000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppStartAttemptEvent.java +++ /dev/null @@ -1,36 +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.hadoop.yarn.server.resourcemanager.rmapp.attempt; - -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; - -public class RMAppStartAttemptEvent extends RMAppAttemptEvent { - - private final boolean transferStateFromPreviousAttempt; - - public RMAppStartAttemptEvent(ApplicationAttemptId appAttemptId, - boolean transferStateFromPreviousAttempt) { - super(appAttemptId, RMAppAttemptEventType.START); - this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt; - } - - public boolean getTransferStateFromPreviousAttempt() { - return transferStateFromPreviousAttempt; - } -} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java index 0dfdf20..bda8580 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java @@ -21,6 +21,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; @@ -50,4 +51,22 @@ Priority getReservedPriority(); + Resource getAllocatedResource(); + + NodeId getAllocatedNode(); + + Priority getAllocatedPriority(); + + long getStartTime(); + + long getFinishTime(); + + String getDiagnosticsInfo(); + + String getLogURL(); + + int getContainerExitStatus(); + + ContainerState getContainerState(); + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java index d44fd3f..057c9ac 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java @@ -28,10 +28,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent; @@ -40,6 +43,7 @@ import org.apache.hadoop.yarn.state.SingleArcTransition; import org.apache.hadoop.yarn.state.StateMachine; import org.apache.hadoop.yarn.state.StateMachineFactory; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils; @SuppressWarnings({"unchecked", "rawtypes"}) public class RMContainerImpl implements RMContainer { @@ -133,28 +137,39 @@ private final ApplicationAttemptId appAttemptId; private final NodeId nodeId; private final Container container; + private final RMContext rmContext; private final EventHandler eventHandler; private final ContainerAllocationExpirer containerAllocationExpirer; + private final String user; private Resource reservedResource; private NodeId reservedNode; private Priority reservedPriority; + private long startTime; + private long finishTime; + private String logURL; + private ContainerStatus finishedStatus; + public RMContainerImpl(Container container, ApplicationAttemptId appAttemptId, NodeId nodeId, - EventHandler handler, - ContainerAllocationExpirer containerAllocationExpirer) { + String user, RMContext rmContext) { this.stateMachine = stateMachineFactory.make(this); this.containerId = container.getId(); this.nodeId = nodeId; this.container = container; this.appAttemptId = appAttemptId; - this.eventHandler = handler; - this.containerAllocationExpirer = containerAllocationExpirer; + this.user = user; + this.startTime = System.currentTimeMillis(); + this.rmContext = rmContext; + this.eventHandler = rmContext.getDispatcher().getEventHandler(); + this.containerAllocationExpirer = rmContext.getContainerAllocationExpirer(); ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); this.readLock = lock.readLock(); this.writeLock = lock.writeLock(); + + rmContext.getRMApplicationHistoryWriter().containerStarted(this); } @Override @@ -197,7 +212,77 @@ public NodeId getReservedNode() { public Priority getReservedPriority() { return reservedPriority; } - + + @Override + public Resource getAllocatedResource() { + return container.getResource(); + } + + @Override + public NodeId getAllocatedNode() { + return container.getNodeId(); + } + + @Override + public Priority getAllocatedPriority() { + return container.getPriority(); + } + + @Override + public long getStartTime() { + return startTime; + } + + @Override + public long getFinishTime() { + try { + readLock.lock(); + return finishTime; + } finally { + readLock.unlock(); + } + } + + @Override + public String getDiagnosticsInfo() { + try { + readLock.lock(); + return finishedStatus.getDiagnostics(); + } finally { + readLock.unlock(); + } + } + + @Override + public String getLogURL() { + try { + readLock.lock(); + return logURL; + } finally { + readLock.unlock(); + } + } + + @Override + public int getContainerExitStatus() { + try { + readLock.lock(); + return finishedStatus.getExitStatus(); + } finally { + readLock.unlock(); + } + } + + @Override + public ContainerState getContainerState() { + try { + readLock.lock(); + return finishedStatus.getState(); + } finally { + readLock.unlock(); + } + } + @Override public String toString() { return containerId.toString(); @@ -276,6 +361,11 @@ public void transition(RMContainerImpl container, RMContainerEvent event) { @Override public void transition(RMContainerImpl container, RMContainerEvent event) { + // The logs of running containers should be found on NM webUI + // The logs should be accessible after the container is launched + container.logURL = WebAppUtils.getLogUrl(container.container + .getNodeHttpAddress(), container.getAllocatedNode().toString(), + container.containerId, container.user); // Unregister from containerAllocationExpirer. container.containerAllocationExpirer.unregister(container .getContainerId()); @@ -288,9 +378,17 @@ public void transition(RMContainerImpl container, RMContainerEvent event) { public void transition(RMContainerImpl container, RMContainerEvent event) { RMContainerFinishedEvent finishedEvent = (RMContainerFinishedEvent) event; + container.finishTime = System.currentTimeMillis(); + container.finishedStatus = finishedEvent.getRemoteContainerStatus(); + // TODO: when AHS webUI is ready, logURL should be updated to point to + // the web page that will show the aggregated logs + // Inform AppAttempt container.eventHandler.handle(new RMAppAttemptContainerFinishedEvent( container.appAttemptId, finishedEvent.getRemoteContainerStatus())); + + container.rmContext.getRMApplicationHistoryWriter() + .containerFinished(container); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java index 4ccd53a..b5b22b6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java @@ -59,10 +59,10 @@ final Set priorities = new TreeSet( new org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.Comparator()); - final Map> requests = + final Map> requests = new HashMap>(); - private Set blacklist = new HashSet(); - + final Set blacklist = new HashSet(); + //private final ApplicationStore store; private final ActiveUsersManager activeUsersManager; @@ -399,15 +399,4 @@ synchronized public void stop(RMAppAttemptState rmAppAttemptFinalState) { public synchronized void setQueue(Queue queue) { this.queue = queue; } - - public synchronized Set getBlackList() { - return this.blacklist; - } - - public synchronized void transferStateFromPreviousAppSchedulingInfo( - AppSchedulingInfo appInfo) { - // this.priorities = appInfo.getPriorities(); - // this.requests = appInfo.getRequests(); - this.blacklist = appInfo.getBlackList(); - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java index eb61785..48e3ee8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplication.java @@ -26,7 +26,6 @@ private final Queue queue; private final String user; - private SchedulerApplicationAttempt currentAttempt; public SchedulerApplication(Queue queue, String user) { this.queue = queue; @@ -40,12 +39,4 @@ public Queue getQueue() { public String getUser() { return user; } - - public SchedulerApplicationAttempt getCurrentAppAttempt() { - return currentAttempt; - } - - public void setCurrentAppAttempt(SchedulerApplicationAttempt currentAttempt) { - this.currentAttempt = currentAttempt; - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java index 446fe84..e5b1cae 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java @@ -64,7 +64,7 @@ protected final AppSchedulingInfo appSchedulingInfo; - protected Map liveContainers = + protected final Map liveContainers = new HashMap(); protected final Map> reservedContainers = new HashMap>(); @@ -73,7 +73,7 @@ protected final Resource currentReservation = Resource.newInstance(0, 0); private Resource resourceLimit = Resource.newInstance(0, 0); - protected Resource currentConsumption = Resource.newInstance(0, 0); + protected final Resource currentConsumption = Resource.newInstance(0, 0); protected List newlyAllocatedContainers = new ArrayList(); @@ -233,8 +233,7 @@ public synchronized RMContainer reserve(SchedulerNode node, Priority priority, if (rmContainer == null) { rmContainer = new RMContainerImpl(container, getApplicationAttemptId(), - node.getNodeID(), rmContext.getDispatcher().getEventHandler(), - rmContext.getContainerAllocationExpirer()); + node.getNodeID(), appSchedulingInfo.getUser(), rmContext); Resources.addTo(currentReservation, container.getResource()); @@ -407,29 +406,4 @@ public synchronized ApplicationResourceUsageReport getResourceUsageReport() { Resources.add(currentConsumption, currentReservation)); } - public synchronized Map getLiveContainersMap() { - return this.liveContainers; - } - - public synchronized Resource getResourceLimit() { - return this.resourceLimit; - } - - public synchronized Map getLastScheduledContainer() { - return this.lastScheduledContainer; - } - - public synchronized void transferStateFromPreviousAttempt( - SchedulerApplicationAttempt appAttempt) { - this.liveContainers = appAttempt.getLiveContainersMap(); - // this.reReservations = appAttempt.reReservations; - this.currentConsumption = appAttempt.getCurrentConsumption(); - this.resourceLimit = appAttempt.getResourceLimit(); - // this.currentReservation = appAttempt.currentReservation; - // this.newlyAllocatedContainers = appAttempt.newlyAllocatedContainers; - // this.schedulingOpportunities = appAttempt.schedulingOpportunities; - this.lastScheduledContainer = appAttempt.getLastScheduledContainer(); - this.appSchedulingInfo - .transferStateFromPreviousAppSchedulingInfo(appAttempt.appSchedulingInfo); - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java index 4f1cb74..b0a56a4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java @@ -19,13 +19,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler; import java.io.IOException; +import java.util.Collection; import java.util.List; import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; @@ -37,7 +37,6 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; /** @@ -171,13 +170,4 @@ boolean checkAccess(UserGroupInformation callerUGI, @LimitedPrivate("yarn") @Stable public List getAppsInQueue(String queueName); - - /** - * Get the container for the given containerId. - * @param containerId - * @return the container for the given containerId. - */ - @LimitedPrivate("yarn") - @Unstable - public RMContainer getRMContainer(ContainerId containerId); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index 4616916..5f34108 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -63,15 +63,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; -import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; @@ -195,6 +194,10 @@ public Configuration getConf() { protected Map applications = new ConcurrentHashMap(); + @VisibleForTesting + protected Map appAttempts = + new ConcurrentHashMap(); + private boolean initialized = false; private ResourceCalculator calculator; @@ -461,27 +464,21 @@ private synchronized void addApplication(ApplicationId applicationId, } private synchronized void addApplicationAttempt( - ApplicationAttemptId applicationAttemptId, - boolean transferStateFromPreviousAttempt) { + ApplicationAttemptId applicationAttemptId) { SchedulerApplication application = applications.get(applicationAttemptId.getApplicationId()); CSQueue queue = (CSQueue) application.getQueue(); - FiCaSchedulerApp attempt = + FiCaSchedulerApp SchedulerApp = new FiCaSchedulerApp(applicationAttemptId, application.getUser(), queue, queue.getActiveUsersManager(), rmContext); - if (transferStateFromPreviousAttempt) { - attempt.transferStateFromPreviousAttempt(application - .getCurrentAppAttempt()); - } - application.setCurrentAppAttempt(attempt); - - queue.submitApplicationAttempt(attempt, application.getUser()); + appAttempts.put(applicationAttemptId, SchedulerApp); + queue.submitApplicationAttempt(SchedulerApp, application.getUser()); LOG.info("Added Application Attempt " + applicationAttemptId + " to scheduler from user " + application.getUser() + " in queue " + queue.getQueueName()); - rmContext.getDispatcher().getEventHandler() .handle( - new RMAppAttemptEvent(applicationAttemptId, + rmContext.getDispatcher().getEventHandler().handle( + new RMAppAttemptEvent(applicationAttemptId, RMAppAttemptEventType.ATTEMPT_ADDED)); } @@ -489,8 +486,7 @@ private synchronized void doneApplication(ApplicationId applicationId, RMAppState finalState) { SchedulerApplication application = applications.get(applicationId); if (application == null){ - // The AppRemovedSchedulerEvent maybe sent on recovery for completed apps, - // ignore it. + // The AppRemovedSchedulerEvent maybe sent on recovery for completed apps. return; } CSQueue queue = (CSQueue) application.getQueue(); @@ -505,56 +501,52 @@ private synchronized void doneApplication(ApplicationId applicationId, private synchronized void doneApplicationAttempt( ApplicationAttemptId applicationAttemptId, - RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) { + RMAppAttemptState rmAppAttemptFinalState) { LOG.info("Application Attempt " + applicationAttemptId + " is done." + " finalState=" + rmAppAttemptFinalState); - FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId); - SchedulerApplication application = - applications.get(applicationAttemptId.getApplicationId()); + FiCaSchedulerApp application = getApplication(applicationAttemptId); - if (application == null || attempt == null) { + if (application == null) { + // throw new IOException("Unknown application " + applicationId + + // " has completed!"); LOG.info("Unknown application " + applicationAttemptId + " has completed!"); return; } - - // Release all the allocated, acquired, running containers - for (RMContainer rmContainer : attempt.getLiveContainers()) { - if (keepContainers - && rmContainer.getState().equals(RMContainerState.RUNNING)) { - // do not kill the running container in the case of work-preserving AM - // restart. - LOG.info("Skip killing " + rmContainer.getContainerId()); - continue; - } - completedContainer( - rmContainer, - SchedulerUtils.createAbnormalContainerStatus( - rmContainer.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION), - RMContainerEventType.KILL); + + // Release all the running containers + for (RMContainer rmContainer : application.getLiveContainers()) { + completedContainer(rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + rmContainer.getContainerId(), + SchedulerUtils.COMPLETED_APPLICATION), + RMContainerEventType.KILL); } - - // Release all reserved containers - for (RMContainer rmContainer : attempt.getReservedContainers()) { - completedContainer( - rmContainer, - SchedulerUtils.createAbnormalContainerStatus( - rmContainer.getContainerId(), "Application Complete"), - RMContainerEventType.KILL); + + // Release all reserved containers + for (RMContainer rmContainer : application.getReservedContainers()) { + completedContainer(rmContainer, + SchedulerUtils.createAbnormalContainerStatus( + rmContainer.getContainerId(), + "Application Complete"), + RMContainerEventType.KILL); } - + // Clean up pending requests, metrics etc. - attempt.stop(rmAppAttemptFinalState); - + application.stop(rmAppAttemptFinalState); + // Inform the queue - String queueName = attempt.getQueue().getQueueName(); + String queueName = application.getQueue().getQueueName(); CSQueue queue = queues.get(queueName); if (!(queue instanceof LeafQueue)) { LOG.error("Cannot finish application " + "from non-leaf queue: " + queueName); } else { - queue.finishApplicationAttempt(attempt, queue.getQueueName()); + queue.finishApplicationAttempt(application, queue.getQueueName()); } + + // Remove from our data-structure + appAttempts.remove(applicationAttemptId); } private static final Allocation EMPTY_ALLOCATION = @@ -566,7 +558,7 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId, List ask, List release, List blacklistAdditions, List blacklistRemovals) { - FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId); + FiCaSchedulerApp application = getApplication(applicationAttemptId); if (application == null) { LOG.info("Calling allocate on removed " + "or non existant application " + applicationAttemptId); @@ -708,8 +700,8 @@ private synchronized void nodeUpdate(RMNode nm) { RMContainer reservedContainer = node.getReservedContainer(); if (reservedContainer != null) { - FiCaSchedulerApp reservedApplication = - getCurrentAttemptForContainer(reservedContainer.getContainerId()); + FiCaSchedulerApp reservedApplication = + getApplication(reservedContainer.getApplicationAttemptId()); // Try to fulfill the reservation LOG.info("Trying to fulfill reservation for application " + @@ -746,11 +738,12 @@ private synchronized void nodeUpdate(RMNode nm) { private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { // Get the application for the finished container - FiCaSchedulerApp application = getCurrentAttemptForContainer(containerId); + ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); + FiCaSchedulerApp application = getApplication(applicationAttemptId); if (application == null) { - LOG.info("Unknown application " - + containerId.getApplicationAttemptId().getApplicationId() - + " launched container " + containerId + " on node: " + node); + LOG.info("Unknown application: " + applicationAttemptId + + " launched container " + containerId + + " on node: " + node); this.rmContext.getDispatcher().getEventHandler() .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId)); return; @@ -798,8 +791,7 @@ public void handle(SchedulerEvent event) { { AppAttemptAddedSchedulerEvent appAttemptAddedEvent = (AppAttemptAddedSchedulerEvent) event; - addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(), - appAttemptAddedEvent.getTransferStateFromPreviousAttempt()); + addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId()); } break; case APP_ATTEMPT_REMOVED: @@ -807,8 +799,7 @@ public void handle(SchedulerEvent event) { AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent = (AppAttemptRemovedSchedulerEvent) event; doneApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(), - appAttemptRemovedEvent.getFinalAttemptState(), - appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts()); + appAttemptRemovedEvent.getFinalAttemptState()); } break; case CONTAINER_EXPIRED: @@ -883,13 +874,13 @@ private synchronized void completedContainer(RMContainer rmContainer, Container container = rmContainer.getContainer(); // Get the application for the finished container - FiCaSchedulerApp application = - getCurrentAttemptForContainer(container.getId()); - ApplicationId appId = - container.getId().getApplicationAttemptId().getApplicationId(); + ApplicationAttemptId applicationAttemptId = + container.getId().getApplicationAttemptId(); + FiCaSchedulerApp application = getApplication(applicationAttemptId); if (application == null) { - LOG.info("Container " + container + " of" + " unknown application " - + appId + " completed with event " + event); + LOG.info("Container " + container + " of" + + " unknown application " + applicationAttemptId + + " completed with event " + event); return; } @@ -901,33 +892,28 @@ private synchronized void completedContainer(RMContainer rmContainer, queue.completedContainer(clusterResource, application, node, rmContainer, containerStatus, event, null); - LOG.info("Application attempt " + application.getApplicationAttemptId() - + " released container " + container.getId() + " on node: " + node - + " with event: " + event); + LOG.info("Application " + applicationAttemptId + + " released container " + container.getId() + + " on node: " + node + + " with event: " + event); } @Lock(Lock.NoLock.class) - FiCaSchedulerApp getApplicationAttempt( - ApplicationAttemptId applicationAttemptId) { - SchedulerApplication app = - applications.get(applicationAttemptId.getApplicationId()); - if (app != null) { - return (FiCaSchedulerApp) app.getCurrentAppAttempt(); - } - return null; + FiCaSchedulerApp getApplication(ApplicationAttemptId applicationAttemptId) { + return appAttempts.get(applicationAttemptId); } @Override public SchedulerAppReport getSchedulerAppInfo( ApplicationAttemptId applicationAttemptId) { - FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId); + FiCaSchedulerApp app = getApplication(applicationAttemptId); return app == null ? null : new SchedulerAppReport(app); } @Override public ApplicationResourceUsageReport getAppResourceUsageReport( ApplicationAttemptId applicationAttemptId) { - FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId); + FiCaSchedulerApp app = getApplication(applicationAttemptId); return app == null ? null : app.getResourceUsageReport(); } @@ -936,22 +922,10 @@ FiCaSchedulerNode getNode(NodeId nodeId) { return nodes.get(nodeId); } - @Override - public RMContainer getRMContainer(ContainerId containerId) { - FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId); - return (attempt == null) ? null : attempt.getRMContainer(containerId); - } - - @VisibleForTesting - public FiCaSchedulerApp getCurrentAttemptForContainer( - ContainerId containerId) { - SchedulerApplication app = - applications.get(containerId.getApplicationAttemptId() - .getApplicationId()); - if (app != null) { - return (FiCaSchedulerApp) app.getCurrentAppAttempt(); - } - return null; + private RMContainer getRMContainer(ContainerId containerId) { + FiCaSchedulerApp application = + getApplication(containerId.getApplicationAttemptId()); + return (application == null) ? null : application.getRMContainer(containerId); } @Override @@ -984,7 +958,7 @@ public void preemptContainer(ApplicationAttemptId aid, RMContainer cont) { LOG.debug("PREEMPT_CONTAINER: application:" + aid.toString() + " container: " + cont.toString()); } - FiCaSchedulerApp app = getApplicationAttempt(aid); + FiCaSchedulerApp app = appAttempts.get(aid); if (app != null) { app.addPreemptContainer(cont.getContainerId()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index dcbc5ad..9c34f2f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -121,9 +121,8 @@ synchronized public RMContainer allocate(NodeType type, FiCaSchedulerNode node, // Create RMContainer RMContainer rmContainer = new RMContainerImpl(container, this - .getApplicationAttemptId(), node.getNodeID(), this.rmContext - .getDispatcher().getEventHandler(), this.rmContext - .getContainerAllocationExpirer()); + .getApplicationAttemptId(), node.getNodeID(), + appSchedulingInfo.getUser(), this.rmContext); // Add it to allContainers list. newlyAllocatedContainers.add(rmContainer); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java index 12ff195..9c5a606 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java @@ -219,8 +219,7 @@ public synchronized void reserveResource( " on node " + this.reservedContainer.getReservedNode()); } - // Cannot reserve more than one application attempt on a given node! - // Reservation is still against attempt. + // Cannot reserve more than one application on a given node! if (!this.reservedContainer.getContainer().getId().getApplicationAttemptId().equals( reservedContainer.getContainer().getId().getApplicationAttemptId())) { throw new IllegalStateException("Trying to reserve" + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptAddedSchedulerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptAddedSchedulerEvent.java index d31010d..d50c157 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptAddedSchedulerEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptAddedSchedulerEvent.java @@ -23,21 +23,14 @@ public class AppAttemptAddedSchedulerEvent extends SchedulerEvent { private final ApplicationAttemptId applicationAttemptId; - private final boolean transferStateFromPreviousAttempt; public AppAttemptAddedSchedulerEvent( - ApplicationAttemptId applicationAttemptId, - boolean transferStateFromPreviousAttempt) { + ApplicationAttemptId applicationAttemptId) { super(SchedulerEventType.APP_ATTEMPT_ADDED); this.applicationAttemptId = applicationAttemptId; - this.transferStateFromPreviousAttempt = transferStateFromPreviousAttempt; } public ApplicationAttemptId getApplicationAttemptId() { return applicationAttemptId; } - - public boolean getTransferStateFromPreviousAttempt() { - return transferStateFromPreviousAttempt; - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptRemovedSchedulerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptRemovedSchedulerEvent.java index 7d944fc..876d164 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptRemovedSchedulerEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/AppAttemptRemovedSchedulerEvent.java @@ -25,15 +25,13 @@ private final ApplicationAttemptId applicationAttemptId; private final RMAppAttemptState finalAttemptState; - private final boolean keepContainersAcrossAppAttempts; public AppAttemptRemovedSchedulerEvent( ApplicationAttemptId applicationAttemptId, - RMAppAttemptState finalAttemptState, boolean keepContainers) { + RMAppAttemptState finalAttemptState) { super(SchedulerEventType.APP_ATTEMPT_REMOVED); this.applicationAttemptId = applicationAttemptId; this.finalAttemptState = finalAttemptState; - this.keepContainersAcrossAppAttempts = keepContainers; } public ApplicationAttemptId getApplicationAttemptID() { @@ -43,8 +41,4 @@ public ApplicationAttemptId getApplicationAttemptID() { public RMAppAttemptState getFinalAttemptState() { return this.finalAttemptState; } - - public boolean getKeepContainersAcrossAppAttempts() { - return this.keepContainersAcrossAppAttempts; - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java index 0bdac8c..adabfef 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerApp.java @@ -271,9 +271,8 @@ else if (allowed.equals(NodeType.RACK_LOCAL) && // Create RMContainer RMContainer rmContainer = new RMContainerImpl(container, - getApplicationAttemptId(), node.getNodeID(), rmContext - .getDispatcher().getEventHandler(), rmContext - .getContainerAllocationExpirer()); + getApplicationAttemptId(), node.getNodeID(), + appSchedulingInfo.getUser(), rmContext); // Add it to allContainers list. newlyAllocatedContainers.add(rmContainer); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java index a009f81..bdfbcab 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java @@ -162,6 +162,12 @@ protected Map applications = new ConcurrentHashMap(); + // This stores per-application-attempt scheduling information, indexed by + // attempt ID's for fast lookup. + @VisibleForTesting + protected Map appAttempts = + new ConcurrentHashMap(); + // Nodes in the cluster, indexed by NodeId private Map nodes = new ConcurrentHashMap(); @@ -256,21 +262,10 @@ public QueueManager getQueueManager() { return queueMgr; } - @Override - public RMContainer getRMContainer(ContainerId containerId) { - FSSchedulerApp attempt = getCurrentAttemptForContainer(containerId); - return (attempt == null) ? null : attempt.getRMContainer(containerId); - } - - private FSSchedulerApp getCurrentAttemptForContainer( - ContainerId containerId) { - SchedulerApplication app = - applications.get(containerId.getApplicationAttemptId() - .getApplicationId()); - if (app != null) { - return (FSSchedulerApp) app.getCurrentAppAttempt(); - } - return null; + private RMContainer getRMContainer(ContainerId containerId) { + FSSchedulerApp application = + appAttempts.get(containerId.getApplicationAttemptId()); + return (application == null) ? null : application.getRMContainer(containerId); } /** @@ -645,8 +640,7 @@ protected synchronized void addApplication(ApplicationId applicationId, applications.put(applicationId, application); LOG.info("Accepted application " + applicationId + " from user: " + user - + ", in queue: " + queueName + ", currently num of applications: " - + applications.size()); + + ", in queue: " + queueName); rmContext.getDispatcher().getEventHandler() .handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED)); } @@ -655,35 +649,31 @@ protected synchronized void addApplication(ApplicationId applicationId, * Add a new application attempt to the scheduler. */ protected synchronized void addApplicationAttempt( - ApplicationAttemptId applicationAttemptId, - boolean transferStateFromPreviousAttempt) { + ApplicationAttemptId applicationAttemptId) { SchedulerApplication application = applications.get(applicationAttemptId.getApplicationId()); String user = application.getUser(); FSLeafQueue queue = (FSLeafQueue) application.getQueue(); - FSSchedulerApp attempt = + FSSchedulerApp schedulerApp = new FSSchedulerApp(applicationAttemptId, user, queue, new ActiveUsersManager(getRootQueueMetrics()), rmContext); - if (transferStateFromPreviousAttempt) { - attempt.transferStateFromPreviousAttempt(application - .getCurrentAppAttempt()); - } - application.setCurrentAppAttempt(attempt); boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user); - queue.addApp(attempt, runnable); + queue.addApp(schedulerApp, runnable); if (runnable) { - maxRunningEnforcer.trackRunnableApp(attempt); + maxRunningEnforcer.trackRunnableApp(schedulerApp); } else { - maxRunningEnforcer.trackNonRunnableApp(attempt); + maxRunningEnforcer.trackNonRunnableApp(schedulerApp); } queue.getMetrics().submitApp(user, applicationAttemptId.getAttemptId()); + appAttempts.put(applicationAttemptId, schedulerApp); LOG.info("Added Application Attempt " + applicationAttemptId - + " to scheduler from user: " + user); + + " to scheduler from user: " + user + ", currently active: " + + appAttempts.size()); rmContext.getDispatcher().getEventHandler().handle( new RMAppAttemptEvent(applicationAttemptId, RMAppAttemptEventType.ATTEMPT_ADDED)); @@ -719,27 +709,19 @@ private synchronized void removeApplication(ApplicationId applicationId, private synchronized void removeApplicationAttempt( ApplicationAttemptId applicationAttemptId, - RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) { + RMAppAttemptState rmAppAttemptFinalState) { LOG.info("Application " + applicationAttemptId + " is done." + " finalState=" + rmAppAttemptFinalState); - SchedulerApplication application = - applications.get(applicationAttemptId.getApplicationId()); - FSSchedulerApp attempt = getSchedulerApp(applicationAttemptId); - if (attempt == null || application == null) { + FSSchedulerApp application = appAttempts.get(applicationAttemptId); + + if (application == null) { LOG.info("Unknown application " + applicationAttemptId + " has completed!"); return; } // Release all the running containers - for (RMContainer rmContainer : attempt.getLiveContainers()) { - if (keepContainers - && rmContainer.getState().equals(RMContainerState.RUNNING)) { - // do not kill the running container in the case of work-preserving AM - // restart. - LOG.info("Skip killing " + rmContainer.getContainerId()); - continue; - } + for (RMContainer rmContainer : application.getLiveContainers()) { completedContainer(rmContainer, SchedulerUtils.createAbnormalContainerStatus( rmContainer.getContainerId(), @@ -748,26 +730,30 @@ private synchronized void removeApplicationAttempt( } // Release all reserved containers - for (RMContainer rmContainer : attempt.getReservedContainers()) { + for (RMContainer rmContainer : application.getReservedContainers()) { completedContainer(rmContainer, SchedulerUtils.createAbnormalContainerStatus( rmContainer.getContainerId(), "Application Complete"), - RMContainerEventType.KILL); + RMContainerEventType.KILL); } + // Clean up pending requests, metrics etc. - attempt.stop(rmAppAttemptFinalState); + application.stop(rmAppAttemptFinalState); // Inform the queue - FSLeafQueue queue = queueMgr.getLeafQueue(attempt.getQueue() + FSLeafQueue queue = queueMgr.getLeafQueue(application.getQueue() .getQueueName(), false); - boolean wasRunnable = queue.removeApp(attempt); + boolean wasRunnable = queue.removeApp(application); if (wasRunnable) { - maxRunningEnforcer.updateRunnabilityOnAppRemoval(attempt); + maxRunningEnforcer.updateRunnabilityOnAppRemoval(application); } else { - maxRunningEnforcer.untrackNonRunnableApp(attempt); + maxRunningEnforcer.untrackNonRunnableApp(application); } + + // Remove from our data-structure + appAttempts.remove(applicationAttemptId); } /** @@ -783,13 +769,11 @@ private synchronized void completedContainer(RMContainer rmContainer, Container container = rmContainer.getContainer(); // Get the application for the finished container - FSSchedulerApp application = - getCurrentAttemptForContainer(container.getId()); - ApplicationId appId = - container.getId().getApplicationAttemptId().getApplicationId(); + ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId(); + FSSchedulerApp application = appAttempts.get(applicationAttemptId); if (application == null) { LOG.info("Container " + container + " of" + - " unknown application attempt " + appId + + " unknown application " + applicationAttemptId + " completed with event " + event); return; } @@ -806,9 +790,10 @@ private synchronized void completedContainer(RMContainer rmContainer, updateRootQueueMetrics(); } - LOG.info("Application attempt " + application.getApplicationAttemptId() - + " released container " + container.getId() + " on node: " + node - + " with event: " + event); + LOG.info("Application " + applicationAttemptId + + " released container " + container.getId() + + " on node: " + node + + " with event: " + event); } private synchronized void addNode(RMNode node) { @@ -859,7 +844,7 @@ public Allocation allocate(ApplicationAttemptId appAttemptId, List ask, List release, List blacklistAdditions, List blacklistRemovals) { // Make sure this application exists - FSSchedulerApp application = getSchedulerApp(appAttemptId); + FSSchedulerApp application = appAttempts.get(appAttemptId); if (application == null) { LOG.info("Calling allocate on removed " + "or non existant application " + appAttemptId); @@ -929,11 +914,12 @@ public Allocation allocate(ApplicationAttemptId appAttemptId, */ private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) { // Get the application for the finished container - FSSchedulerApp application = getCurrentAttemptForContainer(containerId); + ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); + FSSchedulerApp application = appAttempts.get(applicationAttemptId); if (application == null) { - LOG.info("Unknown application " - + containerId.getApplicationAttemptId().getApplicationId() - + " launched container " + containerId + " on node: " + node); + LOG.info("Unknown application: " + applicationAttemptId + + " launched container " + containerId + + " on node: " + node); return; } @@ -1072,34 +1058,28 @@ public SchedulerNodeReport getNodeReport(NodeId nodeId) { } public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) { - SchedulerApplication app = - applications.get(appAttemptId.getApplicationId()); - if (app != null) { - return (FSSchedulerApp) app.getCurrentAppAttempt(); - } - return null; + return appAttempts.get(appAttemptId); } @Override public SchedulerAppReport getSchedulerAppInfo( ApplicationAttemptId appAttemptId) { - FSSchedulerApp attempt = getSchedulerApp(appAttemptId); - if (attempt == null) { + if (!appAttempts.containsKey(appAttemptId)) { LOG.error("Request for appInfo of unknown attempt" + appAttemptId); return null; } - return new SchedulerAppReport(attempt); + return new SchedulerAppReport(appAttempts.get(appAttemptId)); } @Override public ApplicationResourceUsageReport getAppResourceUsageReport( ApplicationAttemptId appAttemptId) { - FSSchedulerApp attempt = getSchedulerApp(appAttemptId); - if (attempt == null) { + FSSchedulerApp app = appAttempts.get(appAttemptId); + if (app == null) { LOG.error("Request for appInfo of unknown attempt" + appAttemptId); return null; } - return attempt.getResourceUsageReport(); + return app.getResourceUsageReport(); } /** @@ -1165,8 +1145,7 @@ public void handle(SchedulerEvent event) { } AppAttemptAddedSchedulerEvent appAttemptAddedEvent = (AppAttemptAddedSchedulerEvent) event; - addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(), - appAttemptAddedEvent.getTransferStateFromPreviousAttempt()); + addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId()); break; case APP_ATTEMPT_REMOVED: if (!(event instanceof AppAttemptRemovedSchedulerEvent)) { @@ -1174,10 +1153,8 @@ public void handle(SchedulerEvent event) { } AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent = (AppAttemptRemovedSchedulerEvent) event; - removeApplicationAttempt( - appAttemptRemovedEvent.getApplicationAttemptID(), - appAttemptRemovedEvent.getFinalAttemptState(), - appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts()); + removeApplicationAttempt(appAttemptRemovedEvent.getApplicationAttemptID(), + appAttemptRemovedEvent.getFinalAttemptState()); break; case CONTAINER_EXPIRED: if (!(event instanceof ContainerExpiredSchedulerEvent)) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index 78e03bd..9d42913 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -67,7 +67,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; -import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo; @@ -124,11 +123,15 @@ private Resource maximumAllocation; private boolean usePortForNodeName; - // Use ConcurrentSkipListMap because applications need to be ordered @VisibleForTesting protected Map applications = new ConcurrentSkipListMap(); + // Use ConcurrentSkipListMap because applications need to be ordered + @VisibleForTesting + protected Map appAttempts + = new ConcurrentSkipListMap(); + private ActiveUsersManager activeUsersManager; private static final String DEFAULT_QUEUE_NAME = "default"; @@ -267,7 +270,7 @@ public Resource getMaximumResourceCapability() { public Allocation allocate( ApplicationAttemptId applicationAttemptId, List ask, List release, List blacklistAdditions, List blacklistRemovals) { - FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId); + FiCaSchedulerApp application = getApplication(applicationAttemptId); if (application == null) { LOG.error("Calling allocate on removed " + "or non existant application " + applicationAttemptId); @@ -333,26 +336,22 @@ public Allocation allocate( } @VisibleForTesting - FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId applicationAttemptId) { - SchedulerApplication app = - applications.get(applicationAttemptId.getApplicationId()); - if (app != null) { - return (FiCaSchedulerApp) app.getCurrentAppAttempt(); - } - return null; + FiCaSchedulerApp getApplication( + ApplicationAttemptId applicationAttemptId) { + return appAttempts.get(applicationAttemptId); } @Override public SchedulerAppReport getSchedulerAppInfo( ApplicationAttemptId applicationAttemptId) { - FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId); + FiCaSchedulerApp app = getApplication(applicationAttemptId); return app == null ? null : new SchedulerAppReport(app); } @Override public ApplicationResourceUsageReport getAppResourceUsageReport( ApplicationAttemptId applicationAttemptId) { - FiCaSchedulerApp app = getApplicationAttempt(applicationAttemptId); + FiCaSchedulerApp app = getApplication(applicationAttemptId); return app == null ? null : app.getResourceUsageReport(); } @@ -365,15 +364,13 @@ private synchronized void addApplication(ApplicationId applicationId, SchedulerApplication application = new SchedulerApplication(null, user); applications.put(applicationId, application); - LOG.info("Accepted application " + applicationId + " from user: " + user - + ", currently num of applications: " + applications.size()); + LOG.info("Accepted application " + applicationId + " from user: " + user); rmContext.getDispatcher().getEventHandler() .handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED)); } - private synchronized void - addApplicationAttempt(ApplicationAttemptId appAttemptId, - boolean transferStateFromPreviousAttempt) { + private synchronized void addApplicationAttempt( + ApplicationAttemptId appAttemptId) { SchedulerApplication application = applications.get(appAttemptId.getApplicationId()); String user = application.getUser(); @@ -381,16 +378,11 @@ private synchronized void addApplication(ApplicationId applicationId, FiCaSchedulerApp schedulerApp = new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE, activeUsersManager, this.rmContext); - - if (transferStateFromPreviousAttempt) { - schedulerApp.transferStateFromPreviousAttempt(application - .getCurrentAppAttempt()); - } - application.setCurrentAppAttempt(schedulerApp); - + appAttempts.put(appAttemptId, schedulerApp); metrics.submitApp(user, appAttemptId.getAttemptId()); LOG.info("Added Application Attempt " + appAttemptId - + " to scheduler from user " + application.getUser()); + + " to scheduler from user " + application.getUser() + + ", currently active: " + appAttempts.size()); rmContext.getDispatcher().getEventHandler().handle( new RMAppAttemptEvent(appAttemptId, RMAppAttemptEventType.ATTEMPT_ADDED)); @@ -408,33 +400,28 @@ private synchronized void doneApplication(ApplicationId applicationId, private synchronized void doneApplicationAttempt( ApplicationAttemptId applicationAttemptId, - RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) + RMAppAttemptState rmAppAttemptFinalState) throws IOException { - FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId); - SchedulerApplication application = - applications.get(applicationAttemptId.getApplicationId()); - if (application == null || attempt == null) { + FiCaSchedulerApp application = getApplication(applicationAttemptId); + if (application == null) { throw new IOException("Unknown application " + applicationAttemptId + " has completed!"); } // Kill all 'live' containers - for (RMContainer container : attempt.getLiveContainers()) { - if (keepContainers - && container.getState().equals(RMContainerState.RUNNING)) { - // do not kill the running container in the case of work-preserving AM - // restart. - LOG.info("Skip killing " + container.getContainerId()); - continue; - } - containerCompleted(container, - SchedulerUtils.createAbnormalContainerStatus( - container.getContainerId(), SchedulerUtils.COMPLETED_APPLICATION), - RMContainerEventType.KILL); + for (RMContainer container : application.getLiveContainers()) { + containerCompleted(container, + SchedulerUtils.createAbnormalContainerStatus( + container.getContainerId(), + SchedulerUtils.COMPLETED_APPLICATION), + RMContainerEventType.KILL); } // Clean up pending requests, metrics etc. - attempt.stop(rmAppAttemptFinalState); + application.stop(rmAppAttemptFinalState); + + // Remove the application + appAttempts.remove(applicationAttemptId); } /** @@ -445,13 +432,12 @@ private synchronized void doneApplicationAttempt( private void assignContainers(FiCaSchedulerNode node) { LOG.debug("assignContainers:" + " node=" + node.getRMNode().getNodeAddress() + - " #applications=" + applications.size()); + " #applications=" + appAttempts.size()); // Try to assign containers to applications in fifo order - for (Map.Entry e : applications + for (Map.Entry e : appAttempts .entrySet()) { - FiCaSchedulerApp application = - (FiCaSchedulerApp) e.getValue().getCurrentAppAttempt(); + FiCaSchedulerApp application = e.getValue(); LOG.debug("pre-assignContainers"); application.showRequests(); synchronized (application) { @@ -488,10 +474,8 @@ private void assignContainers(FiCaSchedulerNode node) { // Update the applications' headroom to correctly take into // account the containers assigned in this update. - for (SchedulerApplication application : applications.values()) { - FiCaSchedulerApp attempt = - (FiCaSchedulerApp) application.getCurrentAppAttempt(); - attempt.setHeadroom(Resources.subtract(clusterResource, usedResource)); + for (FiCaSchedulerApp application : appAttempts.values()) { + application.setHeadroom(Resources.subtract(clusterResource, usedResource)); } } @@ -760,8 +744,7 @@ public void handle(SchedulerEvent event) { { AppAttemptAddedSchedulerEvent appAttemptAddedEvent = (AppAttemptAddedSchedulerEvent) event; - addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(), - appAttemptAddedEvent.getTransferStateFromPreviousAttempt()); + addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId()); } break; case APP_ATTEMPT_REMOVED: @@ -771,8 +754,7 @@ public void handle(SchedulerEvent event) { try { doneApplicationAttempt( appAttemptRemovedEvent.getApplicationAttemptID(), - appAttemptRemovedEvent.getFinalAttemptState(), - appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts()); + appAttemptRemovedEvent.getFinalAttemptState()); } catch(IOException ie) { LOG.error("Unable to remove application " + appAttemptRemovedEvent.getApplicationAttemptID(), ie); @@ -798,11 +780,12 @@ public void handle(SchedulerEvent event) { private void containerLaunchedOnNode(ContainerId containerId, FiCaSchedulerNode node) { // Get the application for the finished container - FiCaSchedulerApp application = getCurrentAttemptForContainer(containerId); + ApplicationAttemptId applicationAttemptId = containerId.getApplicationAttemptId(); + FiCaSchedulerApp application = getApplication(applicationAttemptId); if (application == null) { - LOG.info("Unknown application " - + containerId.getApplicationAttemptId().getApplicationId() - + " launched container " + containerId + " on node: " + node); + LOG.info("Unknown application: " + applicationAttemptId + + " launched container " + containerId + + " on node: " + node); // Some unknown container sneaked into the system. Kill it. this.rmContext.getDispatcher().getEventHandler() .handle(new RMNodeCleanContainerEvent(node.getNodeID(), containerId)); @@ -823,16 +806,14 @@ private synchronized void containerCompleted(RMContainer rmContainer, // Get the application for the finished container Container container = rmContainer.getContainer(); - FiCaSchedulerApp application = - getCurrentAttemptForContainer(container.getId()); - ApplicationId appId = - container.getId().getApplicationAttemptId().getApplicationId(); + ApplicationAttemptId applicationAttemptId = container.getId().getApplicationAttemptId(); + FiCaSchedulerApp application = getApplication(applicationAttemptId); // Get the node on which the container was allocated FiCaSchedulerNode node = getNode(container.getNodeId()); if (application == null) { - LOG.info("Unknown application: " + appId + + LOG.info("Unknown application: " + applicationAttemptId + " released container " + container.getId() + " on node: " + node + " with event: " + event); @@ -848,7 +829,7 @@ private synchronized void containerCompleted(RMContainer rmContainer, // Update total usage Resources.subtractFrom(usedResource, container.getResource()); - LOG.info("Application attempt " + application.getApplicationAttemptId() + + LOG.info("Application " + applicationAttemptId + " released container " + container.getId() + " on node: " + node + " with event: " + event); @@ -906,22 +887,11 @@ public synchronized SchedulerNodeReport getNodeReport(NodeId nodeId) { FiCaSchedulerNode node = getNode(nodeId); return node == null ? null : new SchedulerNodeReport(node); } - - @Override - public RMContainer getRMContainer(ContainerId containerId) { - FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId); - return (attempt == null) ? null : attempt.getRMContainer(containerId); - } - - private FiCaSchedulerApp getCurrentAttemptForContainer( - ContainerId containerId) { - SchedulerApplication app = - applications.get(containerId.getApplicationAttemptId() - .getApplicationId()); - if (app != null) { - return (FiCaSchedulerApp) app.getCurrentAppAttempt(); - } - return null; + + private RMContainer getRMContainer(ContainerId containerId) { + FiCaSchedulerApp application = + getApplication(containerId.getApplicationAttemptId()); + return (application == null) ? null : application.getRMContainer(containerId); } @Override @@ -938,12 +908,12 @@ public synchronized boolean checkAccess(UserGroupInformation callerUGI, @Override public synchronized List getAppsInQueue(String queueName) { if (queueName.equals(DEFAULT_QUEUE.getQueueName())) { - List attempts = new ArrayList( - applications.size()); - for (SchedulerApplication app : applications.values()) { - attempts.add(app.getCurrentAppAttempt().getApplicationAttemptId()); + List apps = new ArrayList( + appAttempts.size()); + for (FiCaSchedulerApp app : appAttempts.values()) { + apps.add(app.getApplicationAttemptId()); } - return attempts; + return apps; } else { return null; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutBlock.java index 91b5cc1..6eed4bf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutBlock.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AboutBlock.java @@ -43,7 +43,6 @@ protected void render(Block html) { info("Cluster overview"). _("Cluster ID:", cinfo.getClusterId()). _("ResourceManager state:", cinfo.getState()). - _("ResourceManager HA state:", cinfo.getHAState()). _("ResourceManager started on:", Times.format(cinfo.getStartedOn())). _("ResourceManager version:", cinfo.getRMBuildVersion() + " on " + cinfo.getRMVersionBuiltOn()). diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java index c96d73e..2da7960 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java @@ -21,7 +21,6 @@ import javax.xml.bind.annotation.XmlAccessorType; import javax.xml.bind.annotation.XmlRootElement; -import org.apache.hadoop.ha.HAServiceProtocol; import org.apache.hadoop.service.Service.STATE; import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; @@ -34,7 +33,6 @@ protected long id; protected long startedOn; protected STATE state; - protected HAServiceProtocol.HAServiceState haState; protected String resourceManagerVersion; protected String resourceManagerBuildVersion; protected String resourceManagerVersionBuiltOn; @@ -50,7 +48,6 @@ public ClusterInfo(ResourceManager rm) { this.id = ts; this.state = rm.getServiceState(); - this.haState = rm.getRMContext().getHAServiceState(); this.startedOn = ts; this.resourceManagerVersion = YarnVersionInfo.getVersion(); this.resourceManagerBuildVersion = YarnVersionInfo.getBuildVersion(); @@ -64,10 +61,6 @@ public String getState() { return this.state.toString(); } - public String getHAState() { - return this.haState.toString(); - } - public String getRMVersion() { return this.resourceManagerVersion; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java index ce5dd96..1192c30 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java @@ -171,7 +171,7 @@ public synchronized void submit() throws IOException, YarnException { new AppAddedSchedulerEvent(this.applicationId, this.queue, "user"); scheduler.handle(addAppEvent); AppAttemptAddedSchedulerEvent addAttemptEvent = - new AppAttemptAddedSchedulerEvent(this.applicationAttemptId, false); + new AppAttemptAddedSchedulerEvent(this.applicationAttemptId); scheduler.handle(addAttemptEvent); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java index 1dcac06..eb69162 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java @@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.YarnVersionInfo; -import org.mortbay.log.Log; public class MockNM { @@ -131,13 +130,12 @@ public NodeHeartbeatResponse nodeHeartbeat(ApplicationAttemptId attemptId, int containerId, ContainerState containerState) throws Exception { HashMap> nodeUpdate = new HashMap>(1); - ContainerStatus containerStatus = BuilderUtils.newContainerStatus( - BuilderUtils.newContainerId(attemptId, containerId), containerState, - "Success", 0); + ContainerStatus amContainerStatus = BuilderUtils.newContainerStatus( + BuilderUtils.newContainerId(attemptId, 1), + ContainerState.COMPLETE, "Success", 0); ArrayList containerStatusList = new ArrayList(1); - containerStatusList.add(containerStatus); - Log.info("ContainerStatus: " + containerStatus); + containerStatusList.add(amContainerStatus); nodeUpdate.put(attemptId.getApplicationId(), containerStatusList); return nodeHeartbeat(nodeUpdate, true); } @@ -154,7 +152,6 @@ public NodeHeartbeatResponse nodeHeartbeat(Map> entry : conts.entrySet()) { - Log.info("entry.getValue() " + entry.getValue()); status.setContainersStatuses(entry.getValue()); } NodeHealthStatus healthStatus = Records.newRecord(NodeHealthStatus.class); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java index 935820e..a480e2b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java @@ -30,7 +30,6 @@ import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest; @@ -41,10 +40,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; -import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.apache.hadoop.yarn.api.records.ContainerState; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.Resource; @@ -60,8 +56,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent; -import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; -import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType; @@ -128,33 +122,6 @@ public void waitForState(ApplicationAttemptId attemptId, attempt.getAppAttemptState()); } - public void waitForContainerAllocated(MockNM nm, ContainerId containerId) - throws Exception { - int timeoutSecs = 0; - while (getResourceScheduler().getRMContainer(containerId) == null - && timeoutSecs++ < 40) { - System.out.println("Waiting for" + containerId + " to be allocated."); - nm.nodeHeartbeat(true); - Thread.sleep(200); - } - } - - public void waitForState(MockNM nm, ContainerId containerId, - RMContainerState containerState) throws Exception { - RMContainer container = getResourceScheduler().getRMContainer(containerId); - Assert.assertNotNull("Container shouldn't be null", container); - int timeoutSecs = 0; - while (!containerState.equals(container.getState()) && timeoutSecs++ < 40) { - System.out.println("Container : " + containerId + " State is : " - + container.getState() + " Waiting for state : " + containerState); - nm.nodeHeartbeat(true); - Thread.sleep(300); - } - System.out.println("Container State is : " + container.getState()); - Assert.assertEquals("Container state is not correct (timedout)", - containerState, container.getState()); - } - // get new application id public GetNewApplicationResponse getNewAppId() throws Exception { ApplicationClientProtocol client = getClientRMService(); @@ -205,17 +172,7 @@ public RMApp submitApp(int masterMemory, String name, String user, public RMApp submitApp(int masterMemory, String name, String user, Map acls, boolean unmanaged, String queue, int maxAppAttempts, Credentials ts, String appType, - boolean waitForAccepted) - throws Exception { - return submitApp(masterMemory, name, user, acls, unmanaged, queue, - maxAppAttempts, ts, appType, waitForAccepted, false); - } - - public RMApp submitApp(int masterMemory, String name, String user, - Map acls, boolean unmanaged, String queue, - int maxAppAttempts, Credentials ts, String appType, - boolean waitForAccepted, boolean keepContainers) - throws Exception { + boolean waitForAccepted) throws Exception { ApplicationClientProtocol client = getClientRMService(); GetNewApplicationResponse resp = client.getNewApplication(Records .newRecord(GetNewApplicationRequest.class)); @@ -225,7 +182,6 @@ public RMApp submitApp(int masterMemory, String name, String user, .newRecord(SubmitApplicationRequest.class); ApplicationSubmissionContext sub = Records .newRecord(ApplicationSubmissionContext.class); - sub.setKeepContainersAcrossApplicationAttempts(keepContainers); sub.setApplicationId(appId); sub.setApplicationName(name); sub.setMaxAppAttempts(maxAppAttempts); @@ -465,26 +421,4 @@ protected void startWepApp() { // override to disable webapp } - public static void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm, - MockAM am) throws Exception { - FinishApplicationMasterRequest req = - FinishApplicationMasterRequest.newInstance( - FinalApplicationStatus.SUCCEEDED, "", ""); - am.unregisterAppAttempt(req); - am.waitForState(RMAppAttemptState.FINISHING); - nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); - am.waitForState(RMAppAttemptState.FINISHED); - rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED); - } - - public static MockAM launchAM(RMApp app, MockRM rm, MockNM nm) - throws Exception { - RMAppAttempt attempt = app.getCurrentAppAttempt(); - nm.nodeHeartbeat(true); - MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId()); - am.registerAppAttempt(); - rm.waitForState(app.getApplicationId(), RMAppState.RUNNING); - return am; - } - } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java index 0a37382..94db331 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java @@ -22,9 +22,9 @@ import static org.mockito.Matchers.isA; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import static org.mockito.Mockito.times; import java.util.HashMap; import java.util.List; @@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; @@ -104,9 +105,10 @@ public static RMContext mockRMContext(int n, long time) { rmDispatcher); AMLivelinessMonitor amFinishingMonitor = new AMLivelinessMonitor( rmDispatcher); + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); RMContext context = new RMContextImpl(rmDispatcher, containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor, - null, null, null, null, null) { + null, null, null, null, null, writer) { @Override public ConcurrentMap getRMApps() { return map; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java index 77398a7..6ef417d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java @@ -19,21 +19,21 @@ package org.apache.hadoop.yarn.server.resourcemanager; import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; import java.io.IOException; import java.net.InetSocketAddress; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashSet; import java.util.EnumSet; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -51,9 +51,9 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.MockApps; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse; @@ -81,6 +81,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; @@ -599,6 +600,8 @@ private void mockRMContext(YarnScheduler yarnScheduler, RMContext rmContext) .thenReturn(queInfo); when(yarnScheduler.getQueueInfo(eq("nonexistentqueue"), anyBoolean(), anyBoolean())) .thenThrow(new IOException("queue does not exist")); + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); + when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer); ConcurrentHashMap apps = getRMApps(rmContext, yarnScheduler); when(rmContext.getRMApps()).thenReturn(apps); @@ -649,7 +652,7 @@ private RMAppImpl getRMApp(RMContext rmContext, YarnScheduler yarnScheduler, .currentTimeMillis(), "YARN")); ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(applicationId3, 1); RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId, - rmContext, yarnScheduler, null, asContext, config, false); + rmContext, yarnScheduler, null, asContext, config); when(app.getCurrentAppAttempt()).thenReturn(rmAppAttemptImpl); return app; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java index cb40ee3..4bf0c44 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java @@ -302,7 +302,7 @@ public void testBlackListNodes() throws Exception { new AppAddedSchedulerEvent(appId1, "queue", "user"); fs.handle(appEvent); SchedulerEvent attemptEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId1, false); + new AppAttemptAddedSchedulerEvent(appAttemptId1); fs.handle(attemptEvent); List emptyId = new ArrayList(); @@ -396,7 +396,7 @@ public void testHeadroom() throws Exception { new AppAddedSchedulerEvent(appId1, "queue", "user"); fs.handle(appEvent); SchedulerEvent attemptEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId1, false); + new AppAttemptAddedSchedulerEvent(appAttemptId1); fs.handle(attemptEvent); ApplicationId appId2 = BuilderUtils.newApplicationId(200, 2); @@ -406,7 +406,7 @@ public void testHeadroom() throws Exception { new AppAddedSchedulerEvent(appId2, "queue", "user"); fs.handle(appEvent2); SchedulerEvent attemptEvent2 = - new AppAttemptAddedSchedulerEvent(appAttemptId2, false); + new AppAttemptAddedSchedulerEvent(appAttemptId2); fs.handle(attemptEvent2); List emptyId = new ArrayList(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java index 32932a5..a2bf4ae 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; +import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; @@ -37,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.NMToken; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.Token; @@ -293,8 +295,6 @@ public void testNMToken() throws Exception { nm2.nodeHeartbeat(attempt.getAppAttemptId(), container.getId().getId(), ContainerState.COMPLETE); } - nm1.nodeHeartbeat(am.getApplicationAttemptId(), 1, - ContainerState.COMPLETE); am.waitForState(RMAppAttemptState.FINISHED); Assert.assertFalse(nmTokenSecretManager .isApplicationAttemptRegistered(attempt.getAppAttemptId())); @@ -389,19 +389,19 @@ public void testInvalidateAMHostPortWhenAMFailedOrKilled() throws Exception { MockNM nm1 = new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService()); nm1.registerNode(); - MockAM am1 = MockRM.launchAM(app1, rm1, nm1); - MockRM.finishApplicationMaster(app1, rm1, nm1, am1); + MockAM am1 = launchAM(app1, rm1, nm1); + finishApplicationMaster(app1, rm1, nm1, am1); // a failed app RMApp app2 = rm1.submitApp(200); - MockAM am2 = MockRM.launchAM(app2, rm1, nm1); + MockAM am2 = launchAM(app2, rm1, nm1); nm1.nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE); am2.waitForState(RMAppAttemptState.FAILED); rm1.waitForState(app2.getApplicationId(), RMAppState.FAILED); // a killed app RMApp app3 = rm1.submitApp(200); - MockAM am3 = MockRM.launchAM(app3, rm1, nm1); + MockAM am3 = launchAM(app3, rm1, nm1); rm1.killApp(app3.getApplicationId()); rm1.waitForState(app3.getApplicationId(), RMAppState.KILLED); rm1.waitForState(am3.getApplicationAttemptId(), RMAppAttemptState.KILLED); @@ -441,7 +441,7 @@ public void testInvalidatedAMHostPortOnAMRestart() throws Exception { // a failed app RMApp app2 = rm1.submitApp(200); - MockAM am2 = MockRM.launchAM(app2, rm1, nm1); + MockAM am2 = launchAM(app2, rm1, nm1); nm1 .nodeHeartbeat(am2.getApplicationAttemptId(), 1, ContainerState.COMPLETE); am2.waitForState(RMAppAttemptState.FAILED); @@ -458,6 +458,28 @@ public void testInvalidatedAMHostPortOnAMRestart() throws Exception { Assert.assertEquals(-1, report1.getRpcPort()); } + private MockAM launchAM(RMApp app, MockRM rm, MockNM nm) + throws Exception { + RMAppAttempt attempt = app.getCurrentAppAttempt(); + nm.nodeHeartbeat(true); + MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId()); + am.registerAppAttempt(); + rm.waitForState(app.getApplicationId(), RMAppState.RUNNING); + return am; + } + + private void finishApplicationMaster(RMApp rmApp, MockRM rm, MockNM nm, + MockAM am) throws Exception { + FinishApplicationMasterRequest req = + FinishApplicationMasterRequest.newInstance( + FinalApplicationStatus.SUCCEEDED, "", ""); + am.unregisterAppAttempt(req); + am.waitForState(RMAppAttemptState.FINISHING); + nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); + am.waitForState(RMAppAttemptState.FINISHED); + rm.waitForState(rmApp.getApplicationId(), RMAppState.FINISHED); + } + public static void main(String[] args) throws Exception { TestRM t = new TestRM(); t.testGetNewAppId(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java index 82046c7..a966efd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java @@ -101,7 +101,7 @@ public void setUp() throws Exception { rmContext = new RMContextImpl(rmDispatcher, null, null, null, - mock(DelegationTokenRenewer.class), null, null, null, null); + mock(DelegationTokenRenewer.class), null, null, null, null, null); scheduler = mock(YarnScheduler.class); doAnswer( new Answer() { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java index 3a113c9..1f9d179 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java @@ -164,7 +164,7 @@ public void testResourceAllocation() throws IOException, // Notify scheduler application is finished. AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent( - application.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false); + application.getApplicationAttemptId(), RMAppAttemptState.FINISHED); resourceManager.getResourceScheduler().handle(appRemovedEvent1); checkResourceUsage(nm1, nm2); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java new file mode 100644 index 0000000..819bcba --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java @@ -0,0 +1,509 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.ahs; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.event.AsyncDispatcher; +import org.apache.hadoop.yarn.event.Dispatcher; +import org.apache.hadoop.yarn.event.Event; +import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; +import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore; +import org.apache.hadoop.yarn.server.applicationhistoryservice.MemoryApplicationHistoryStore; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationHistoryData; +import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ContainerHistoryData; +import org.apache.hadoop.yarn.server.resourcemanager.MockAM; +import org.apache.hadoop.yarn.server.resourcemanager.MockNM; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestRMApplicationHistoryWriter { + + private static int MAX_RETRIES = 10; + + private RMApplicationHistoryWriter writer; + private ApplicationHistoryStore store; + private List dispatchers = + new ArrayList(); + + @Before + public void setup() { + store = new MemoryApplicationHistoryStore(); + Configuration conf = new Configuration(); + writer = new RMApplicationHistoryWriter() { + + @Override + protected ApplicationHistoryStore createApplicationHistoryStore( + Configuration conf) { + return store; + } + + @Override + protected Dispatcher createDispatcher(Configuration conf) { + MultiThreadedDispatcher dispatcher = new MultiThreadedDispatcher(conf.getInt( + YarnConfiguration.RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE, + YarnConfiguration.DEFAULT_RM_HISTORY_WRITER_MULTI_THREADED_DISPATCHER_POOL_SIZE)); + dispatcher.setDrainEventsOnStop(); + return dispatcher; + } + + class MultiThreadedDispatcher extends + RMApplicationHistoryWriter.MultiThreadedDispatcher { + + public MultiThreadedDispatcher(int num) { + super(num); + } + + @Override + protected AsyncDispatcher createDispatcher() { + CounterDispatcher dispatcher = new CounterDispatcher(); + dispatchers.add(dispatcher); + return dispatcher; + } + + } + }; + writer.init(conf); + writer.start(); + } + + @After + public void tearDown() { + writer.stop(); + } + + private static RMApp createRMApp(ApplicationId appId) { + RMApp app = mock(RMApp.class); + when(app.getApplicationId()).thenReturn(appId); + when(app.getName()).thenReturn("test app"); + when(app.getApplicationType()).thenReturn("test app type"); + when(app.getUser()).thenReturn("test user"); + when(app.getQueue()).thenReturn("test queue"); + when(app.getSubmitTime()).thenReturn(0L); + when(app.getStartTime()).thenReturn(1L); + when(app.getFinishTime()).thenReturn(2L); + when(app.getDiagnostics()).thenReturn( + new StringBuilder("test diagnostics info")); + when(app.getFinalApplicationStatus()).thenReturn( + FinalApplicationStatus.UNDEFINED); + when(app.createApplicationState()) + .thenReturn(YarnApplicationState.FINISHED); + return app; + } + + private static RMAppAttempt createRMAppAttempt( + ApplicationAttemptId appAttemptId) { + RMAppAttempt appAttempt = mock(RMAppAttempt.class); + when(appAttempt.getAppAttemptId()).thenReturn(appAttemptId); + when(appAttempt.getHost()).thenReturn("test host"); + when(appAttempt.getRpcPort()).thenReturn(-100); + Container container = mock(Container.class); + when(container.getId()).thenReturn( + ContainerId.newInstance(appAttemptId, 1)); + when(appAttempt.getMasterContainer()).thenReturn(container); + when(appAttempt.getDiagnostics()).thenReturn("test diagnostics info"); + when(appAttempt.getTrackingUrl()).thenReturn("test url"); + when(appAttempt.getFinalApplicationStatus()).thenReturn( + FinalApplicationStatus.UNDEFINED); + when(appAttempt.createApplicationAttemptState()).thenReturn( + YarnApplicationAttemptState.FINISHED); + return appAttempt; + } + + private static RMContainer createRMContainer( + ContainerId containerId) { + RMContainer container = mock(RMContainer.class); + when(container.getContainerId()).thenReturn(containerId); + when(container.getAllocatedNode()).thenReturn( + NodeId.newInstance("test host", -100)); + when(container.getAllocatedResource()).thenReturn( + Resource.newInstance(-1, -1)); + when(container.getAllocatedPriority()).thenReturn(Priority.UNDEFINED); + when(container.getStartTime()).thenReturn(0L); + when(container.getFinishTime()).thenReturn(1L); + when(container.getDiagnosticsInfo()).thenReturn("test diagnostics info"); + when(container.getLogURL()).thenReturn("test log url"); + when(container.getContainerExitStatus()).thenReturn(-1); + when(container.getContainerState()).thenReturn(ContainerState.COMPLETE); + return container; + } + + @Test + public void testWriteApplication() throws Exception { + RMApp app = createRMApp(ApplicationId.newInstance(0, 1)); + + writer.applicationStarted(app); + ApplicationHistoryData appHD = null; + for (int i = 0; i < MAX_RETRIES; ++i) { + appHD = store.getApplication(ApplicationId.newInstance(0, 1)); + if (appHD != null) { + break; + } else { + Thread.sleep(100); + } + } + Assert.assertNotNull(appHD); + Assert.assertEquals("test app", appHD.getApplicationName()); + Assert.assertEquals("test app type", appHD.getApplicationType()); + Assert.assertEquals("test user", appHD.getUser()); + Assert.assertEquals("test queue", appHD.getQueue()); + Assert.assertEquals(0L, appHD.getSubmitTime()); + Assert.assertEquals(1L, appHD.getStartTime()); + + writer.applicationFinished(app); + for (int i = 0; i < MAX_RETRIES; ++i) { + appHD = store.getApplication(ApplicationId.newInstance(0, 1)); + if (appHD.getYarnApplicationState() != null) { + break; + } else { + Thread.sleep(100); + } + } + Assert.assertEquals(2L, appHD.getFinishTime()); + Assert.assertEquals("test diagnostics info", appHD.getDiagnosticsInfo()); + Assert.assertEquals(FinalApplicationStatus.UNDEFINED, + appHD.getFinalApplicationStatus()); + Assert.assertEquals(YarnApplicationState.FINISHED, + appHD.getYarnApplicationState()); + } + + @Test + public void testWriteApplicationAttempt() throws Exception { + RMAppAttempt appAttempt = createRMAppAttempt( + ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1)); + writer.applicationAttemptStarted(appAttempt); + ApplicationAttemptHistoryData appAttemptHD = null; + for (int i = 0; i < MAX_RETRIES; ++i) { + appAttemptHD = + store.getApplicationAttempt(ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1)); + if (appAttemptHD != null) { + break; + } else { + Thread.sleep(100); + } + } + Assert.assertNotNull(appAttemptHD); + Assert.assertEquals("test host", appAttemptHD.getHost()); + Assert.assertEquals(-100, appAttemptHD.getRPCPort()); + Assert.assertEquals( + ContainerId.newInstance(ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1), 1), + appAttemptHD.getMasterContainerId()); + + writer.applicationAttemptFinished(appAttempt); + for (int i = 0; i < MAX_RETRIES; ++i) { + appAttemptHD = + store.getApplicationAttempt(ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1)); + if (appAttemptHD.getYarnApplicationAttemptState() != null) { + break; + } else { + Thread.sleep(100); + } + } + Assert.assertEquals("test diagnostics info", + appAttemptHD.getDiagnosticsInfo()); + Assert.assertEquals("test url", appAttemptHD.getTrackingURL()); + Assert.assertEquals(FinalApplicationStatus.UNDEFINED, + appAttemptHD.getFinalApplicationStatus()); + Assert.assertEquals(YarnApplicationAttemptState.FINISHED, + appAttemptHD.getYarnApplicationAttemptState()); + } + + @Test + public void testWriteContainer() throws Exception { + RMContainer container = createRMContainer( + ContainerId.newInstance(ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1), 1)); + writer.containerStarted(container); + ContainerHistoryData containerHD = null; + for (int i = 0; i < MAX_RETRIES; ++i) { + containerHD = + store.getContainer(ContainerId.newInstance( + ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1), 1)); + if (containerHD != null) { + break; + } else { + Thread.sleep(100); + } + } + Assert.assertNotNull(containerHD); + Assert.assertEquals(NodeId.newInstance("test host", -100), + containerHD.getAssignedNode()); + Assert.assertEquals(Resource.newInstance(-1, -1), + containerHD.getAllocatedResource()); + Assert.assertEquals(Priority.UNDEFINED, containerHD.getPriority()); + Assert.assertEquals(0L, container.getStartTime()); + + writer.containerFinished(container); + for (int i = 0; i < MAX_RETRIES; ++i) { + containerHD = + store.getContainer(ContainerId.newInstance( + ApplicationAttemptId.newInstance( + ApplicationId.newInstance(0, 1), 1), 1)); + if (containerHD.getContainerState() != null) { + break; + } else { + Thread.sleep(100); + } + } + Assert.assertEquals("test diagnostics info", + containerHD.getDiagnosticsInfo()); + Assert.assertEquals("test log url", containerHD.getLogURL()); + Assert.assertEquals(-1, containerHD.getContainerExitStatus()); + Assert.assertEquals(ContainerState.COMPLETE, + containerHD.getContainerState()); + } + + @Test + public void testParallelWrite() throws Exception { + List appIds = new ArrayList(); + for (int i = 0; i < 10; ++i) { + Random rand = new Random(i); + ApplicationId appId = ApplicationId.newInstance(0, rand.nextInt()); + appIds.add(appId); + RMApp app = createRMApp(appId); + writer.applicationStarted(app); + for (int j = 1; j <= 10; ++j) { + ApplicationAttemptId appAttemptId = + ApplicationAttemptId.newInstance(appId, j); + RMAppAttempt appAttempt = createRMAppAttempt(appAttemptId); + writer.applicationAttemptStarted(appAttempt); + for (int k = 1; k <= 10; ++k) { + ContainerId containerId = ContainerId.newInstance(appAttemptId, k); + RMContainer container = createRMContainer(containerId); + writer.containerStarted(container); + writer.containerFinished(container); + } + writer.applicationAttemptFinished(appAttempt); + } + writer.applicationFinished(app); + } + for (int i = 0; i < MAX_RETRIES; ++i) { + if (allEventsHandled(20 * 10 * 10 + 20 * 10 + 20)) { + break; + } else { + Thread.sleep(500); + } + } + Assert.assertTrue(allEventsHandled(20 * 10 * 10 + 20 * 10 + 20)); + // Validate all events of one application are handled by one dispatcher + for (ApplicationId appId : appIds) { + Assert.assertTrue(handledByOne(appId)); + } + } + + private boolean allEventsHandled(int expected) { + int actual = 0; + for (CounterDispatcher dispatcher : dispatchers) { + for (Integer count : dispatcher.counts.values()) { + actual += count; + } + } + return actual == expected; + } + + @Test + public void testRMWritingMassiveHistory() throws Exception { + // 1. Show RM can run with writing history data + // 2. Test additional workload of processing history events + YarnConfiguration conf = new YarnConfiguration(); + // don't process history events + MockRM rm = new MockRM(conf) { + @Override + protected RMApplicationHistoryWriter createRMApplicationHistoryWriter() { + return new RMApplicationHistoryWriter() { + @Override + public void applicationStarted(RMApp app) { + } + @Override + public void applicationFinished(RMApp app) { + } + @Override + public void applicationAttemptStarted(RMAppAttempt appAttempt) { + } + @Override + public void applicationAttemptFinished(RMAppAttempt appAttempt) { + } + @Override + public void containerStarted(RMContainer container) { + } + @Override + public void containerFinished(RMContainer container) { + } + }; + } + }; + long startTime1 = System.currentTimeMillis(); + testRMWritingMassiveHistory(rm); + long finishTime1 = System.currentTimeMillis(); + long elapsedTime1 = finishTime1 - startTime1; + rm = new MockRM(conf); + long startTime2 = System.currentTimeMillis(); + testRMWritingMassiveHistory(rm); + long finishTime2 = System.currentTimeMillis(); + long elapsedTime2 = finishTime2 - startTime2; + // No more than 10% additional workload + // Should be much less, but computation time is fluctuated + Assert.assertTrue(elapsedTime2 - elapsedTime1 < elapsedTime1 / 10); + } + + private void testRMWritingMassiveHistory(MockRM rm) throws Exception { + rm.start(); + MockNM nm = rm.registerNode("127.0.0.1:1234", 1024 * 10100); + + RMApp app = rm.submitApp(1024); + nm.nodeHeartbeat(true); + RMAppAttempt attempt = app.getCurrentAppAttempt(); + MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId()); + am.registerAppAttempt(); + + int request = 10000; + am.allocate("127.0.0.1" , 1024, request, + new ArrayList()); + nm.nodeHeartbeat(true); + List allocated = am.allocate(new ArrayList(), + new ArrayList()).getAllocatedContainers(); + int waitCount = 0; + int allocatedSize = allocated.size(); + while (allocatedSize < request && waitCount++ < 200) { + Thread.sleep(100); + allocated = am.allocate(new ArrayList(), + new ArrayList()).getAllocatedContainers(); + allocatedSize += allocated.size(); + nm.nodeHeartbeat(true); + } + Assert.assertEquals(request, allocatedSize); + + am.unregisterAppAttempt(); + am.waitForState(RMAppAttemptState.FINISHING); + nm.nodeHeartbeat(am.getApplicationAttemptId(), 1, ContainerState.COMPLETE); + am.waitForState(RMAppAttemptState.FINISHED); + + NodeHeartbeatResponse resp = nm.nodeHeartbeat(true); + List cleaned = resp.getContainersToCleanup(); + int cleanedSize = cleaned.size(); + waitCount = 0; + while (cleanedSize < allocatedSize && waitCount++ < 200) { + Thread.sleep(100); + resp = nm.nodeHeartbeat(true); + cleaned = resp.getContainersToCleanup(); + cleanedSize += cleaned.size(); + } + Assert.assertEquals(allocatedSize, cleanedSize); + rm.waitForState(app.getApplicationId(), RMAppState.FINISHED); + + rm.stop(); + } + + private boolean handledByOne(ApplicationId appId) { + int count = 0; + for (CounterDispatcher dispatcher : dispatchers) { + if (dispatcher.counts.containsKey(appId)) { + ++count; + } + } + return count == 1; + } + + private static class CounterDispatcher extends AsyncDispatcher { + + private Map counts = + new HashMap(); + + @SuppressWarnings("rawtypes") + @Override + protected void dispatch(Event event) { + if (event instanceof WritingApplicationHistoryEvent) { + WritingApplicationHistoryEvent ashEvent = + (WritingApplicationHistoryEvent) event; + switch (ashEvent.getType()) { + case APP_START: + incrementCounts(((WritingApplicationStartEvent) event).getApplicationId()); + break; + case APP_FINISH: + incrementCounts(((WritingApplicationFinishEvent) event) + .getApplicationId()); + break; + case APP_ATTEMPT_START: + incrementCounts(((WritingApplicationAttemptStartEvent) event) + .getApplicationAttemptId().getApplicationId()); + break; + case APP_ATTEMPT_FINISH: + incrementCounts(((WritingApplicationAttemptFinishEvent) event) + .getApplicationAttemptId().getApplicationId()); + break; + case CONTAINER_START: + incrementCounts(((WritingContainerStartEvent) event).getContainerId() + .getApplicationAttemptId().getApplicationId()); + break; + case CONTAINER_FINISH: + incrementCounts(((WritingContainerFinishEvent) event).getContainerId() + .getApplicationAttemptId().getApplicationId()); + break; + } + } + super.dispatch(event); + } + + private void incrementCounts(ApplicationId appId) { + Integer val = counts.get(appId); + if (val == null) { + counts.put(appId, 1); + } else { + counts.put(appId, val + 1); + } + } + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java index 4d38e7c..8832769 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java @@ -18,30 +18,49 @@ package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager; -import java.util.ArrayList; -import java.util.HashMap; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; import junit.framework.Assert; -import org.apache.hadoop.yarn.api.records.ApplicationAccessType; -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerState; -import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.MockAM; -import org.apache.hadoop.yarn.server.resourcemanager.MockNM; -import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; -import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; +import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.factories.RecordFactory; +import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; +import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager; +import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.junit.After; +import org.junit.Before; import org.junit.Test; /** @@ -49,164 +68,238 @@ * */ public class TestAMRestart { - - @Test - public void testAMRestartWithExistingContainers() throws Exception { - YarnConfiguration conf = new YarnConfiguration(); - conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2); - - MockRM rm1 = new MockRM(conf); - rm1.start(); - RMApp app1 = - rm1.submitApp(200, "name", "user", - new HashMap(), false, "default", -1, - null, "MAPREDUCE", false, true); - MockNM nm1 = - new MockNM("127.0.0.1:1234", 10240, rm1.getResourceTrackerService()); - nm1.registerNode(); - MockNM nm2 = - new MockNM("127.0.0.1:2351", 4089, rm1.getResourceTrackerService()); - nm2.registerNode(); - - MockAM am1 = MockRM.launchAM(app1, rm1, nm1); - int NUM_CONTAINERS = 3; - // allocate NUM_CONTAINERS containers - am1.allocate("127.0.0.1", 1024, NUM_CONTAINERS, - new ArrayList()); - nm1.nodeHeartbeat(true); - - // wait for containers to be allocated. - List containers = - am1.allocate(new ArrayList(), - new ArrayList()).getAllocatedContainers(); - while (containers.size() != NUM_CONTAINERS) { - nm1.nodeHeartbeat(true); - containers.addAll(am1.allocate(new ArrayList(), - new ArrayList()).getAllocatedContainers()); - Thread.sleep(200); - } - - // launch the 2nd container, for testing running container transferred. - nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING); - ContainerId containerId2 = - ContainerId.newInstance(am1.getApplicationAttemptId(), 2); - rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING); - - // launch the 3rd container, for testing container allocated by previous - // attempt is completed by the next new attempt/ - nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.RUNNING); - ContainerId containerId3 = - ContainerId.newInstance(am1.getApplicationAttemptId(), 3); - rm1.waitForState(nm1, containerId3, RMContainerState.RUNNING); - - // 4th container still in AQUIRED state. for testing Acquired container is - // always killed. - ContainerId containerId4 = - ContainerId.newInstance(am1.getApplicationAttemptId(), 4); - rm1.waitForState(nm1, containerId4, RMContainerState.ACQUIRED); - - // 5th container is in Allocated state. for testing allocated container is - // always killed. - am1.allocate("127.0.0.1", 1024, 1, new ArrayList()); - nm1.nodeHeartbeat(true); - ContainerId containerId5 = - ContainerId.newInstance(am1.getApplicationAttemptId(), 5); - rm1.waitForContainerAllocated(nm1, containerId5); - rm1.waitForState(nm1, containerId5, RMContainerState.ALLOCATED); - - // 6th container is in Reserved state. - am1.allocate("127.0.0.1", 6000, 1, new ArrayList()); - ContainerId containerId6 = - ContainerId.newInstance(am1.getApplicationAttemptId(), 6); - nm1.nodeHeartbeat(true); - SchedulerApplicationAttempt schedulerAttempt = - ((CapacityScheduler) rm1.getResourceScheduler()) - .getCurrentAttemptForContainer(containerId6); - while (schedulerAttempt.getReservedContainers().size() == 0) { - System.out.println("Waiting for container " + containerId6 - + " to be reserved."); - nm1.nodeHeartbeat(true); - Thread.sleep(200); - } - // assert containerId6 is reserved. - Assert.assertEquals(containerId6, schedulerAttempt.getReservedContainers() - .get(0).getContainerId()); - - // fail the AM by sending CONTAINER_FINISHED event without registering. - nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 1, ContainerState.COMPLETE); - am1.waitForState(RMAppAttemptState.FAILED); - - // wait for some time. previous AM's running containers should still remain - // in scheduler even though am failed - Thread.sleep(3000); - rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING); - // acquired/allocated containers are cleaned up. - Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId4)); - Assert.assertNull(rm1.getResourceScheduler().getRMContainer(containerId5)); - - // wait for app to start a new attempt. - rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); - // assert this is a new AM. - ApplicationAttemptId newAttemptId = - app1.getCurrentAppAttempt().getAppAttemptId(); - Assert.assertFalse(newAttemptId.equals(am1.getApplicationAttemptId())); - MockAM am2 = MockRM.launchAM(app1, rm1, nm1); - - // complete container by sending the container complete event which has earlier - // attempt's attemptId - nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.COMPLETE); - rm1.waitForState(nm1, containerId3, RMContainerState.COMPLETED); - - // Even though the completed container containerId3 event was sent to the - // earlier failed attempt, new RMAppAttempt can also capture this container - // info. - // completed containerId4 is also transferred to the new attempt. - RMAppAttempt newAttempt = - app1.getRMAppAttempt(am2.getApplicationAttemptId()); - // 4 containers finished, acquired/allocated/reserved/completed. - Assert.assertEquals(4, newAttempt.getJustFinishedContainers().size()); - boolean container3Exists = false, container4Exists = false, container5Exists = - false, container6Exists = false; - for(ContainerStatus status : newAttempt.getJustFinishedContainers()) { - if(status.getContainerId().equals(containerId3)) { - // containerId3 is the container ran by previous attempt but finished by the - // new attempt. - container3Exists = true; - } - if (status.getContainerId().equals(containerId4)) { - // containerId4 is the Acquired Container killed by the previous attempt, - // it's now inside new attempt's finished container list. - container4Exists = true; - } - if (status.getContainerId().equals(containerId5)) { - // containerId5 is the Allocated container killed by previous failed attempt. - container5Exists = true; - } - if (status.getContainerId().equals(containerId6)) { - // containerId6 is the reserved container killed by previous failed attempt. - container6Exists = true; - } - } - Assert.assertTrue(container3Exists && container4Exists && container5Exists - && container6Exists); - - // New SchedulerApplicationAttempt also has the containers info. - rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING); - - // record the scheduler attempt for testing. - SchedulerApplicationAttempt schedulerNewAttempt = - ((CapacityScheduler) rm1.getResourceScheduler()) - .getCurrentAttemptForContainer(containerId2); - // finish this application - MockRM.finishApplicationMaster(app1, rm1, nm1, am2); - - // the 2nd attempt released the 1st attempt's running container, when the - // 2nd attempt finishes. - Assert.assertFalse(schedulerNewAttempt.getLiveContainers().contains( - containerId2)); - // all 4 normal containers finished. - Assert.assertEquals(5, newAttempt.getJustFinishedContainers().size()); - - rm1.stop(); - } +// private static final Log LOG = LogFactory.getLog(TestAMRestart.class); +// ApplicationsManagerImpl appImpl; +// RMContext asmContext = new RMContextImpl(new MemStore()); +// ApplicationTokenSecretManager appTokenSecretManager = +// new ApplicationTokenSecretManager(); +// DummyResourceScheduler scheduler; +// private ClientRMService clientRMService; +// int count = 0; +// ApplicationId appID; +// final int maxFailures = 3; +// AtomicInteger launchNotify = new AtomicInteger(); +// AtomicInteger schedulerNotify = new AtomicInteger(); +// volatile boolean stop = false; +// int schedulerAddApplication = 0; +// int schedulerRemoveApplication = 0; +// int launcherLaunchCalled = 0; +// int launcherCleanupCalled = 0; +// private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); +// +// private class ExtApplicationsManagerImpl extends ApplicationsManagerImpl { +// public ExtApplicationsManagerImpl( +// ApplicationTokenSecretManager applicationTokenSecretManager, +// YarnScheduler scheduler, RMContext asmContext) { +// super(applicationTokenSecretManager, scheduler, asmContext); +// } +// +// @Override +// public EventHandler> createNewApplicationMasterLauncher( +// ApplicationTokenSecretManager tokenSecretManager) { +// return new DummyAMLauncher(); +// } +// } +// +// private class DummyAMLauncher implements EventHandler> { +// +// public DummyAMLauncher() { +// asmContext.getDispatcher().register(AMLauncherEventType.class, this); +// new Thread() { +// public void run() { +// while (!stop) { +// LOG.info("DEBUG -- waiting for launch"); +// synchronized(launchNotify) { +// while (launchNotify.get() == 0) { +// try { +// launchNotify.wait(); +// } catch (InterruptedException e) { +// } +// } +// asmContext.getDispatcher().getEventHandler().handle( +// new ApplicationEvent( +// ApplicationEventType.LAUNCHED, appID)); +// launchNotify.addAndGet(-1); +// } +// } +// } +// }.start(); +// } +// +// @Override +// public void handle(ASMEvent event) { +// switch (event.getType()) { +// case CLEANUP: +// launcherCleanupCalled++; +// break; +// case LAUNCH: +// LOG.info("DEBUG -- launching"); +// launcherLaunchCalled++; +// synchronized (launchNotify) { +// launchNotify.addAndGet(1); +// launchNotify.notify(); +// } +// break; +// default: +// break; +// } +// } +// } +// +// private class DummyResourceScheduler implements ResourceScheduler { +// +// @Override +// public void removeNode(RMNode node) { +// } +// +// @Override +// public Allocation allocate(ApplicationId applicationId, +// List ask, List release) throws IOException { +// Container container = recordFactory.newRecordInstance(Container.class); +// container.setContainerToken(recordFactory.newRecordInstance(ContainerToken.class)); +// container.setNodeId(recordFactory.newRecordInstance(NodeId.class)); +// container.setContainerManagerAddress("localhost"); +// container.setNodeHttpAddress("localhost:8042"); +// container.setId(recordFactory.newRecordInstance(ContainerId.class)); +// container.getId().setAppId(appID); +// container.getId().setId(count); +// count++; +// return new Allocation(Arrays.asList(container), Resources.none()); +// } +// +// @Override +// public void handle(ASMEvent event) { +// switch (event.getType()) { +// case ADD: +// schedulerAddApplication++; +// break; +// case EXPIRE: +// schedulerRemoveApplication++; +// LOG.info("REMOVING app : " + schedulerRemoveApplication); +// if (schedulerRemoveApplication == maxFailures) { +// synchronized (schedulerNotify) { +// schedulerNotify.addAndGet(1); +// schedulerNotify.notify(); +// } +// } +// break; +// default: +// break; +// } +// } +// +// @Override +// public QueueInfo getQueueInfo(String queueName, +// boolean includeChildQueues, +// boolean recursive) throws IOException { +// return null; +// } +// @Override +// public List getQueueUserAclInfo() { +// return null; +// } +// @Override +// public void addApplication(ApplicationId applicationId, +// ApplicationMaster master, String user, String queue, Priority priority, +// ApplicationStore store) +// throws IOException { +// } +// @Override +// public void addNode(RMNode nodeInfo) { +// } +// @Override +// public void recover(RMState state) throws Exception { +// } +// @Override +// public void reinitialize(Configuration conf, +// ContainerTokenSecretManager secretManager, RMContext rmContext) +// throws IOException { +// } +// +// @Override +// public void nodeUpdate(RMNode nodeInfo, +// Map> containers) { +// } +// +// @Override +// public Resource getMaximumResourceCapability() { +// // TODO Auto-generated method stub +// return null; +// } +// +// @Override +// public Resource getMinimumResourceCapability() { +// // TODO Auto-generated method stub +// return null; +// } +// } +// +// @Before +// public void setUp() { +// +// asmContext.getDispatcher().register(ApplicationEventType.class, +// new ResourceManager.ApplicationEventDispatcher(asmContext)); +// +// appID = recordFactory.newRecordInstance(ApplicationId.class); +// appID.setClusterTimestamp(System.currentTimeMillis()); +// appID.setId(1); +// Configuration conf = new Configuration(); +// scheduler = new DummyResourceScheduler(); +// asmContext.getDispatcher().init(conf); +// asmContext.getDispatcher().start(); +// asmContext.getDispatcher().register(ApplicationTrackerEventType.class, scheduler); +// appImpl = new ExtApplicationsManagerImpl(appTokenSecretManager, scheduler, asmContext); +// +// conf.setLong(YarnConfiguration.AM_EXPIRY_INTERVAL, 1000L); +// conf.setInt(RMConfig.AM_MAX_RETRIES, maxFailures); +// appImpl.init(conf); +// appImpl.start(); +// +// this.clientRMService = new ClientRMService(asmContext, appImpl +// .getAmLivelinessMonitor(), appImpl.getClientToAMSecretManager(), +// scheduler); +// this.clientRMService.init(conf); +// } +// +// @After +// public void tearDown() { +// } +// +// private void waitForFailed(AppAttempt application, ApplicationState +// finalState) throws Exception { +// int count = 0; +// while(application.getState() != finalState && count < 10) { +// Thread.sleep(500); +// count++; +// } +// Assert.assertEquals(finalState, application.getState()); +// } +// +// @Test +// public void testAMRestart() throws Exception { +// ApplicationSubmissionContext subContext = recordFactory.newRecordInstance(ApplicationSubmissionContext.class); +// subContext.setApplicationId(appID); +// subContext.setApplicationName("dummyApp"); +//// subContext.command = new ArrayList(); +//// subContext.environment = new HashMap(); +//// subContext.fsTokens = new ArrayList(); +// subContext.setFsTokensTodo(ByteBuffer.wrap(new byte[0])); +// SubmitApplicationRequest request = recordFactory +// .newRecordInstance(SubmitApplicationRequest.class); +// request.setApplicationSubmissionContext(subContext); +// clientRMService.submitApplication(request); +// AppAttempt application = asmContext.getApplications().get(appID); +// synchronized (schedulerNotify) { +// while(schedulerNotify.get() == 0) { +// schedulerNotify.wait(); +// } +// } +// Assert.assertEquals(maxFailures, launcherCleanupCalled); +// Assert.assertEquals(maxFailures, launcherLaunchCalled); +// Assert.assertEquals(maxFailures, schedulerAddApplication); +// Assert.assertEquals(maxFailures, schedulerRemoveApplication); +// Assert.assertEquals(maxFailures, application.getFailedCount()); +// waitForFailed(application, ApplicationState.FAILED); +// stop = true; +// } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java index a884552..756bf45 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java @@ -72,7 +72,7 @@ public void setUp() { // Dispatcher that processes events inline Dispatcher dispatcher = new InlineDispatcher(); RMContext context = new RMContextImpl(dispatcher, null, - null, null, null, null, null, null, null); + null, null, null, null, null, null, null, null); dispatcher.register(SchedulerEventType.class, new InlineDispatcher.EmptyEventHandler()); dispatcher.register(RMNodeEventType.class, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java index cbb2374..455fa78 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMReconnect.java @@ -71,7 +71,7 @@ public void setUp() { new TestRMNodeEventDispatcher()); RMContext context = new RMContextImpl(dispatcher, null, - null, null, null, null, null, null, null); + null, null, null, null, null, null, null, null); dispatcher.register(SchedulerEventType.class, new InlineDispatcher.EmptyEventHandler()); dispatcher.register(RMNodeEventType.class, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java index ddb7a90..4f94695 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java @@ -71,7 +71,7 @@ public void handle(Event event) { RMContext context = new RMContextImpl(dispatcher, null, null, null, null, null, new RMContainerTokenSecretManager(conf), - new NMTokenSecretManagerInRM(conf), null); + new NMTokenSecretManagerInRM(conf), null, null); dispatcher.register(RMNodeEventType.class, new ResourceManager.NodeEventDispatcher(context)); NodesListManager nodesListManager = new NodesListManager(context); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java index 1e5733b..9e1588b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java @@ -20,6 +20,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -49,6 +50,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor; @@ -83,6 +85,7 @@ private static int appId = 1; private DrainDispatcher rmDispatcher; private RMStateStore store; + private RMApplicationHistoryWriter writer; private YarnScheduler scheduler; // ignore all the RM application attempt events @@ -178,13 +181,15 @@ public void setUp() throws Exception { AMLivelinessMonitor amLivelinessMonitor = mock(AMLivelinessMonitor.class); AMLivelinessMonitor amFinishingMonitor = mock(AMLivelinessMonitor.class); store = mock(RMStateStore.class); + writer = mock(RMApplicationHistoryWriter.class); this.rmContext = new RMContextImpl(rmDispatcher, containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor, null, new AMRMTokenSecretManager(conf), new RMContainerTokenSecretManager(conf), new NMTokenSecretManagerInRM(conf), - new ClientToAMTokenSecretManagerInRM()); + new ClientToAMTokenSecretManagerInRM(), + writer); ((RMContextImpl)rmContext).setStateStore(store); rmDispatcher.register(RMAppAttemptEventType.class, @@ -335,6 +340,7 @@ private void sendAttemptUpdateSavedEvent(RMApp application) { protected RMApp testCreateAppNewSaving( ApplicationSubmissionContext submissionContext) throws IOException { RMApp application = createNewTestApp(submissionContext); + verify(writer).applicationStarted(any(RMApp.class)); // NEW => NEW_SAVING event RMAppEventType.START RMAppEvent event = new RMAppEvent(application.getApplicationId(), RMAppEventType.START); @@ -456,11 +462,14 @@ public void testUnmanagedApp() throws IOException { Assert.assertTrue("Finished app missing diagnostics", application.getDiagnostics().indexOf(diagMsg) != -1); + // reset the counter of Mockito.verify + reset(writer); + // test app fails after 1 app attempt failure LOG.info("--- START: testUnmanagedAppFailPath ---"); application = testCreateAppRunning(subContext); RMAppEvent event = new RMAppFailedAttemptEvent( - application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, "", false); + application.getApplicationId(), RMAppEventType.ATTEMPT_FAILED, ""); application.handle(event); rmDispatcher.await(); RMAppAttempt appAttempt = application.getCurrentAppAttempt(); @@ -497,6 +506,7 @@ public void testAppNewKill() throws IOException { rmDispatcher.await(); sendAppUpdateSavedEvent(application); assertKilled(application); + verify(writer).applicationFinished(any(RMApp.class)); } @Test @@ -512,6 +522,7 @@ public void testAppNewReject() throws IOException { rmDispatcher.await(); sendAppUpdateSavedEvent(application); assertFailed(application, rejectedText); + verify(writer).applicationFinished(any(RMApp.class)); } @Test (timeout = 30000) @@ -526,6 +537,7 @@ public void testAppNewSavingKill() throws IOException { rmDispatcher.await(); sendAppUpdateSavedEvent(application); assertKilled(application); + verify(writer).applicationFinished(any(RMApp.class)); } @Test (timeout = 30000) @@ -541,6 +553,7 @@ public void testAppNewSavingReject() throws IOException { rmDispatcher.await(); sendAppUpdateSavedEvent(application); assertFailed(application, rejectedText); + verify(writer).applicationFinished(any(RMApp.class)); } @Test (timeout = 30000) @@ -556,6 +569,7 @@ public void testAppSubmittedRejected() throws IOException { rmDispatcher.await(); sendAppUpdateSavedEvent(application); assertFailed(application, rejectedText); + verify(writer).applicationFinished(any(RMApp.class)); } @Test @@ -570,6 +584,7 @@ public void testAppSubmittedKill() throws IOException, InterruptedException { sendAppUpdateSavedEvent(application); assertKilled(application); assertAppFinalStateSaved(application); + verify(writer).applicationFinished(any(RMApp.class)); } @Test @@ -582,7 +597,7 @@ public void testAppAcceptedFailed() throws IOException { for (int i=1; i < maxAppAttempts; i++) { RMAppEvent event = new RMAppFailedAttemptEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_FAILED, "", false); + RMAppEventType.ATTEMPT_FAILED, ""); application.handle(event); assertAppState(RMAppState.ACCEPTED, application); event = @@ -598,11 +613,12 @@ public void testAppAcceptedFailed() throws IOException { String message = "Test fail"; RMAppEvent event = new RMAppFailedAttemptEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_FAILED, message, false); + RMAppEventType.ATTEMPT_FAILED, message); application.handle(event); rmDispatcher.await(); sendAppUpdateSavedEvent(application); assertFailed(application, ".*" + message + ".*Failing the application.*"); + verify(writer).applicationFinished(any(RMApp.class)); } @Test @@ -617,6 +633,7 @@ public void testAppAcceptedKill() throws IOException, InterruptedException { sendAppUpdateSavedEvent(application); assertKilled(application); assertAppFinalStateSaved(application); + verify(writer).applicationFinished(any(RMApp.class)); } @Test @@ -639,6 +656,7 @@ public void testAppRunningKill() throws IOException { sendAttemptUpdateSavedEvent(application); sendAppUpdateSavedEvent(application); assertKilled(application); + verify(writer).applicationFinished(any(RMApp.class)); } @Test @@ -655,7 +673,7 @@ public void testAppRunningFailed() throws IOException { for (int i=1; i KILLED event RMAppEventType.ATTEMPT_FAILED event = new RMAppFailedAttemptEvent(application.getApplicationId(), - RMAppEventType.ATTEMPT_FAILED, "", false); + RMAppEventType.ATTEMPT_FAILED, ""); application.handle(event); rmDispatcher.await(); assertTimesAtFinish(application); @@ -820,6 +841,7 @@ public void testAppKilledKilled() throws IOException { assertTimesAtFinish(application); assertAppState(RMAppState.KILLED, application); + verify(writer).applicationFinished(any(RMApp.class)); } @Test diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java index 4286950..d1890d9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java @@ -51,7 +51,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; @@ -64,15 +63,16 @@ import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType; import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent; -import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAcquiredEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent; @@ -119,17 +119,18 @@ private ApplicationMasterLauncher applicationMasterLauncher; private AMLivelinessMonitor amLivelinessMonitor; private AMLivelinessMonitor amFinishingMonitor; + private RMApplicationHistoryWriter writer; + private RMStateStore store; - private RMAppImpl application; + private RMApp application; private RMAppAttempt applicationAttempt; private Configuration conf = new Configuration(); private AMRMTokenSecretManager amRMTokenManager = spy(new AMRMTokenSecretManager(conf)); private ClientToAMTokenSecretManagerInRM clientToAMTokenManager = spy(new ClientToAMTokenSecretManagerInRM()); - private boolean transferStateFromPreviousAttempt = false; - + private final class TestApplicationAttemptEventDispatcher implements EventHandler { @@ -152,11 +153,6 @@ public void handle(RMAppAttemptEvent event) { @Override public void handle(RMAppEvent event) { assertEquals(application.getApplicationId(), event.getApplicationId()); - if (event instanceof RMAppFailedAttemptEvent) { - transferStateFromPreviousAttempt = - ((RMAppFailedAttemptEvent) event) - .getTransferStateFromPreviousAttempt(); - } try { application.handle(event); } catch (Throwable t) { @@ -213,13 +209,15 @@ public void setUp() throws Exception { mock(ContainerAllocationExpirer.class); amLivelinessMonitor = mock(AMLivelinessMonitor.class); amFinishingMonitor = mock(AMLivelinessMonitor.class); + writer = mock(RMApplicationHistoryWriter.class); rmContext = new RMContextImpl(rmDispatcher, containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor, null, amRMTokenManager, new RMContainerTokenSecretManager(conf), new NMTokenSecretManagerInRM(conf), - clientToAMTokenManager); + clientToAMTokenManager, + writer); store = mock(RMStateStore.class); ((RMContextImpl) rmContext).setStateStore(store); @@ -261,10 +259,10 @@ public void setUp() throws Exception { unmanagedAM = false; - application = mock(RMAppImpl.class); + application = mock(RMApp.class); applicationAttempt = new RMAppAttemptImpl(applicationAttemptId, rmContext, scheduler, - masterService, submissionContext, new Configuration(), false); + masterService, submissionContext, new Configuration()); when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt); when(application.getApplicationId()).thenReturn(applicationId); @@ -377,8 +375,8 @@ private void testAppAttemptKilledState(Container amContainer, assertEquals(0, applicationAttempt.getRanNodes().size()); assertNull(applicationAttempt.getFinalApplicationStatus()); verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); + verify(writer).applicationAttemptFinished(any(RMAppAttempt.class)); verifyAttemptFinalStateSaved(); - assertFalse(transferStateFromPreviousAttempt); } /** @@ -452,6 +450,7 @@ private void testAppAttemptFailedState(Container container, // Check events verify(application, times(1)).handle(any(RMAppFailedAttemptEvent.class)); verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); + verify(writer).applicationAttemptFinished(any(RMAppAttempt.class)); verifyAttemptFinalStateSaved(); } @@ -487,6 +486,7 @@ private void testAppAttemptRunningState(Container container, assertEquals(getProxyUrl(applicationAttempt), applicationAttempt.getTrackingUrl()); } + verify(writer).applicationAttemptStarted(any(RMAppAttempt.class)); // TODO - need to add more checks relevant to this state } @@ -533,7 +533,7 @@ private void testAppAttemptFinishedState(Container container, assertEquals(container, applicationAttempt.getMasterContainer()); assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus()); verifyTokenCount(applicationAttempt.getAppAttemptId(), 1); - assertFalse(transferStateFromPreviousAttempt); + verify(writer).applicationAttemptFinished(any(RMAppAttempt.class)); } @@ -663,7 +663,6 @@ private void testUnmanagedAMSuccess(String url) { diagnostics)); testAppAttemptFinishedState(null, finalStatus, url, diagnostics, 1, true); - assertFalse(transferStateFromPreviousAttempt); } private void sendAttemptUpdateSavedEvent(RMAppAttempt applicationAttempt) { @@ -692,21 +691,6 @@ public void testUnmanagedAMUnexpectedRegistration() { } @Test - public void testUnmanagedAMContainersCleanup() { - unmanagedAM = true; - when(submissionContext.getUnmanagedAM()).thenReturn(true); - when(submissionContext.getKeepContainersAcrossApplicationAttempts()) - .thenReturn(true); - // submit AM and check it goes to SUBMITTED state - submitApplicationAttempt(); - // launch AM and verify attempt failed - applicationAttempt.handle(new RMAppAttemptRegistrationEvent( - applicationAttempt.getAppAttemptId(), "host", 8042, "oldtrackingurl")); - sendAttemptUpdateSavedEvent(applicationAttempt); - assertFalse(transferStateFromPreviousAttempt); - } - - @Test public void testNewToKilled() { applicationAttempt.handle( new RMAppAttemptEvent( @@ -1117,64 +1101,6 @@ public void testGetClientToken() throws Exception { Assert.assertNull(token); } - @Test - public void testFailedToFailed() { - // create a failed attempt. - when(submissionContext.getKeepContainersAcrossApplicationAttempts()) - .thenReturn(true); - Container amContainer = allocateApplicationAttempt(); - launchApplicationAttempt(amContainer); - runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false); - ContainerStatus cs1 = - ContainerStatus.newInstance(amContainer.getId(), - ContainerState.COMPLETE, "some error", 123); - ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId(); - applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( - appAttemptId, cs1)); - sendAttemptUpdateSavedEvent(applicationAttempt); - assertEquals(RMAppAttemptState.FAILED, - applicationAttempt.getAppAttemptState()); - // should not kill containers when attempt fails. - assertTrue(transferStateFromPreviousAttempt); - - // failed attempt captured the container finished event. - assertEquals(0, applicationAttempt.getJustFinishedContainers().size()); - ContainerStatus cs2 = - ContainerStatus.newInstance(ContainerId.newInstance(appAttemptId, 2), - ContainerState.COMPLETE, "", 0); - applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( - appAttemptId, cs2)); - assertEquals(1, applicationAttempt.getJustFinishedContainers().size()); - assertEquals(cs2.getContainerId(), applicationAttempt - .getJustFinishedContainers().get(0).getContainerId()); - } - - - @Test - public void testContainersCleanupForLastAttempt() { - // create a failed attempt. - applicationAttempt = - new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), rmContext, - scheduler, masterService, submissionContext, new Configuration(), - true); - when(submissionContext.getKeepContainersAcrossApplicationAttempts()) - .thenReturn(true); - when(submissionContext.getMaxAppAttempts()).thenReturn(1); - Container amContainer = allocateApplicationAttempt(); - launchApplicationAttempt(amContainer); - runApplicationAttempt(amContainer, "host", 8042, "oldtrackingurl", false); - ContainerStatus cs1 = - ContainerStatus.newInstance(amContainer.getId(), - ContainerState.COMPLETE, "some error", 123); - ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId(); - applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent( - appAttemptId, cs1)); - sendAttemptUpdateSavedEvent(applicationAttempt); - assertEquals(RMAppAttemptState.FAILED, - applicationAttempt.getAppAttemptState()); - assertFalse(transferStateFromPreviousAttempt); - } - private void verifyTokenCount(ApplicationAttemptId appAttemptId, int count) { verify(amRMTokenManager, times(count)).applicationMasterFinished(appAttemptId); if (UserGroupInformation.isSecurityEnabled()) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java index d1262d8..4050493 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java @@ -19,14 +19,19 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer; import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; @@ -34,6 +39,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.event.EventHandler; +import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent; @@ -50,7 +57,6 @@ public void testReleaseWhileRunning() { DrainDispatcher drainDispatcher = new DrainDispatcher(); - EventHandler eventHandler = drainDispatcher.getEventHandler(); EventHandler appAttemptEventHandler = mock(EventHandler.class); EventHandler generic = mock(EventHandler.class); drainDispatcher.register(RMAppAttemptEventType.class, @@ -71,16 +77,24 @@ public void testReleaseWhileRunning() { Container container = BuilderUtils.newContainer(containerId, nodeId, "host:3465", resource, priority, null); + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); + RMContext rmContext = mock(RMContext.class); + when(rmContext.getDispatcher()).thenReturn(drainDispatcher); + when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer); + when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer); RMContainer rmContainer = new RMContainerImpl(container, appAttemptId, - nodeId, eventHandler, expirer); + nodeId, "user", rmContext); assertEquals(RMContainerState.NEW, rmContainer.getState()); + assertEquals(resource, rmContainer.getAllocatedResource()); + assertEquals(nodeId, rmContainer.getAllocatedNode()); + assertEquals(priority, rmContainer.getAllocatedPriority()); + verify(writer).containerStarted(any(RMContainer.class)); rmContainer.handle(new RMContainerEvent(containerId, RMContainerEventType.START)); drainDispatcher.await(); assertEquals(RMContainerState.ALLOCATED, rmContainer.getState()); - rmContainer.handle(new RMContainerEvent(containerId, RMContainerEventType.ACQUIRED)); drainDispatcher.await(); @@ -90,6 +104,9 @@ public void testReleaseWhileRunning() { RMContainerEventType.LAUNCHED)); drainDispatcher.await(); assertEquals(RMContainerState.RUNNING, rmContainer.getState()); + assertEquals( + "http://host:3465/node/containerlogs/container_1_0001_01_000001/user", + rmContainer.getLogURL()); // In RUNNING state. Verify RELEASED and associated actions. reset(appAttemptEventHandler); @@ -100,6 +117,12 @@ public void testReleaseWhileRunning() { containerStatus, RMContainerEventType.RELEASED)); drainDispatcher.await(); assertEquals(RMContainerState.RELEASED, rmContainer.getState()); + assertEquals(SchedulerUtils.RELEASED_CONTAINER, + rmContainer.getDiagnosticsInfo()); + assertEquals(ContainerExitStatus.ABORTED, + rmContainer.getContainerExitStatus()); + assertEquals(ContainerState.COMPLETE, rmContainer.getContainerState()); + verify(writer).containerFinished(any(RMContainer.class)); ArgumentCaptor captor = ArgumentCaptor .forClass(RMAppAttemptContainerFinishedEvent.class); @@ -120,7 +143,6 @@ public void testReleaseWhileRunning() { public void testExpireWhileRunning() { DrainDispatcher drainDispatcher = new DrainDispatcher(); - EventHandler eventHandler = drainDispatcher.getEventHandler(); EventHandler appAttemptEventHandler = mock(EventHandler.class); EventHandler generic = mock(EventHandler.class); drainDispatcher.register(RMAppAttemptEventType.class, @@ -141,10 +163,19 @@ public void testExpireWhileRunning() { Container container = BuilderUtils.newContainer(containerId, nodeId, "host:3465", resource, priority, null); + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); + RMContext rmContext = mock(RMContext.class); + when(rmContext.getDispatcher()).thenReturn(drainDispatcher); + when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer); + when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer); RMContainer rmContainer = new RMContainerImpl(container, appAttemptId, - nodeId, eventHandler, expirer); + nodeId, "user", rmContext); assertEquals(RMContainerState.NEW, rmContainer.getState()); + assertEquals(resource, rmContainer.getAllocatedResource()); + assertEquals(nodeId, rmContainer.getAllocatedNode()); + assertEquals(priority, rmContainer.getAllocatedPriority()); + verify(writer).containerStarted(any(RMContainer.class)); rmContainer.handle(new RMContainerEvent(containerId, RMContainerEventType.START)); @@ -160,6 +191,9 @@ public void testExpireWhileRunning() { RMContainerEventType.LAUNCHED)); drainDispatcher.await(); assertEquals(RMContainerState.RUNNING, rmContainer.getState()); + assertEquals( + "http://host:3465/node/containerlogs/container_1_0001_01_000001/user", + rmContainer.getLogURL()); // In RUNNING state. Verify EXPIRE and associated actions. reset(appAttemptEventHandler); @@ -170,5 +204,6 @@ public void testExpireWhileRunning() { containerStatus, RMContainerEventType.EXPIRE)); drainDispatcher.await(); assertEquals(RMContainerState.RUNNING, rmContainer.getState()); + verify(writer, never()).containerFinished(any(RMContainer.class)); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java index fbd7ec7..75ef36a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java @@ -348,7 +348,7 @@ public void testRefreshQueues() throws Exception { cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null, null, new RMContainerTokenSecretManager(conf), new NMTokenSecretManagerInRM(conf), - new ClientToAMTokenSecretManagerInRM())); + new ClientToAMTokenSecretManagerInRM(), null)); checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY); conf.setCapacity(A, 80f); @@ -447,7 +447,7 @@ public void testParseQueue() throws IOException { cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null, null, new RMContainerTokenSecretManager(conf), new NMTokenSecretManagerInRM(conf), - new ClientToAMTokenSecretManagerInRM())); + new ClientToAMTokenSecretManagerInRM(), null)); } @Test @@ -460,7 +460,7 @@ public void testReconnectedNode() throws Exception { cs.reinitialize(csConf, new RMContextImpl(null, null, null, null, null, null, new RMContainerTokenSecretManager(csConf), new NMTokenSecretManagerInRM(csConf), - new ClientToAMTokenSecretManagerInRM())); + new ClientToAMTokenSecretManagerInRM(), null)); RMNode n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1); RMNode n2 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 2); @@ -487,7 +487,7 @@ public void testRefreshQueuesWithNewQueue() throws Exception { cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null, null, new RMContainerTokenSecretManager(conf), new NMTokenSecretManagerInRM(conf), - new ClientToAMTokenSecretManagerInRM())); + new ClientToAMTokenSecretManagerInRM(), null)); checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY); // Add a new queue b4 @@ -562,18 +562,18 @@ public void testBlackListNodes() throws Exception { new AppAddedSchedulerEvent(appId, "default", "user"); cs.handle(addAppEvent); SchedulerEvent addAttemptEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId, false); + new AppAttemptAddedSchedulerEvent(appAttemptId); cs.handle(addAttemptEvent); // Verify the blacklist can be updated independent of requesting containers cs.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), Collections.singletonList(host), null); - Assert.assertTrue(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host)); + Assert.assertTrue(cs.getApplication(appAttemptId).isBlacklisted(host)); cs.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), null, Collections.singletonList(host)); - Assert.assertFalse(cs.getApplicationAttempt(appAttemptId).isBlacklisted(host)); + Assert.assertFalse(cs.getApplication(appAttemptId).isBlacklisted(host)); rm.stop(); } @@ -597,6 +597,66 @@ public void testApplicationComparator() assertTrue(appComparator.compare(app1, app3) < 0); assertTrue(appComparator.compare(app2, app3) < 0); } + + @Test + public void testConcurrentAccessOnApplications() throws Exception { + CapacityScheduler cs = new CapacityScheduler(); + verifyConcurrentAccessOnApplications( + cs.appAttempts, FiCaSchedulerApp.class, Queue.class); + } + + public static + void verifyConcurrentAccessOnApplications( + final Map applications, Class appClazz, + final Class queueClazz) + throws Exception { + final int size = 10000; + final ApplicationId appId = ApplicationId.newInstance(0, 0); + final Constructor ctor = appClazz.getDeclaredConstructor( + ApplicationAttemptId.class, String.class, queueClazz, + ActiveUsersManager.class, RMContext.class); + + ApplicationAttemptId appAttemptId0 + = ApplicationAttemptId.newInstance(appId, 0); + applications.put(appAttemptId0, ctor.newInstance( + appAttemptId0, null, mock(queueClazz), null, null)); + assertNotNull(applications.get(appAttemptId0)); + + // Imitating the thread of scheduler that will add and remove apps + final AtomicBoolean finished = new AtomicBoolean(false); + final AtomicBoolean failed = new AtomicBoolean(false); + Thread t = new Thread() { + + @Override + public void run() { + for (int i = 1; i <= size; ++i) { + ApplicationAttemptId appAttemptId + = ApplicationAttemptId.newInstance(appId, i); + try { + applications.put(appAttemptId, ctor.newInstance( + appAttemptId, null, mock(queueClazz), null, null)); + } catch (Exception e) { + failed.set(true); + finished.set(true); + return; + } + } + for (int i = 1; i <= size; ++i) { + ApplicationAttemptId appAttemptId + = ApplicationAttemptId.newInstance(appId, i); + applications.remove(appAttemptId); + } + finished.set(true); + } + }; + t.start(); + + // Imitating the thread of rmappattempt that will get the app + while (!finished.get()) { + assertNotNull(applications.get(appAttemptId0)); + } + assertFalse(failed.get()); + } @Test public void testGetAppsInQueue() throws Exception { @@ -638,7 +698,7 @@ public void testAddAndRemoveAppFromCapacityScheduler() throws Exception { cs.reinitialize(conf, new RMContextImpl(rmDispatcher, null, null, null, null, null, new RMContainerTokenSecretManager(conf), new NMTokenSecretManagerInRM(conf), - new ClientToAMTokenSecretManagerInRM())); + new ClientToAMTokenSecretManagerInRM(), null)); SchedulerApplication app = TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java index 3c55b42..d509771 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java @@ -41,8 +41,8 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.event.DrainDispatcher; -import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; @@ -248,14 +248,18 @@ public void testSortedQueues() throws Exception { ContainerAllocationExpirer expirer = mock(ContainerAllocationExpirer.class); DrainDispatcher drainDispatcher = new DrainDispatcher(); - EventHandler eventHandler = drainDispatcher.getEventHandler(); + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); + RMContext rmContext = mock(RMContext.class); + when(rmContext.getContainerAllocationExpirer()).thenReturn(expirer); + when(rmContext.getDispatcher()).thenReturn(drainDispatcher); + when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer); ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId( app_0.getApplicationId(), 1); ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1); Container container=TestUtils.getMockContainer(containerId, node_0.getNodeID(), Resources.createResource(1*GB), priority); RMContainer rmContainer = new RMContainerImpl(container, appAttemptId, - node_0.getNodeID(), eventHandler, expirer); + node_0.getNodeID(), "user", rmContext); // Assign {1,2,3,4} 1GB containers respectively to queues stubQueueAllocation(a, clusterResource, node_0, 1*GB); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java index dde1ff4..5e272de 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java @@ -63,8 +63,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; @@ -347,16 +345,11 @@ public void testAppAttemptMetrics() throws Exception { .getMockApplicationAttemptId(0, 1); FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a, null, rmContext); - AppAddedSchedulerEvent addAppEvent = - new AppAddedSchedulerEvent(appAttemptId_0.getApplicationId(), - a.getQueueName(), user_0); - cs.handle(addAppEvent); - AppAttemptAddedSchedulerEvent addAttemptEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId_0, false); - cs.handle(addAttemptEvent); - + a.submitApplicationAttempt(app_0, user_0); + + when(cs.getApplication(appAttemptId_0)).thenReturn(app_0); AppAttemptRemovedSchedulerEvent event = new AppAttemptRemovedSchedulerEvent( - appAttemptId_0, RMAppAttemptState.FAILED, false); + appAttemptId_0, RMAppAttemptState.FAILED); cs.handle(event); assertEquals(0, a.getMetrics().getAppsPending()); @@ -372,8 +365,9 @@ public void testAppAttemptMetrics() throws Exception { assertEquals(1, a.getMetrics().getAppsSubmitted()); assertEquals(1, a.getMetrics().getAppsPending()); + when(cs.getApplication(appAttemptId_1)).thenReturn(app_0); event = new AppAttemptRemovedSchedulerEvent(appAttemptId_0, - RMAppAttemptState.FINISHED, false); + RMAppAttemptState.FINISHED); cs.handle(event); assertEquals(1, a.getMetrics().getAppsSubmitted()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java index c86d6b3..21c446a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueParsing.java @@ -47,7 +47,7 @@ public void testQueueParsing() throws Exception { capacityScheduler.reinitialize(conf, new RMContextImpl(null, null, null, null, null, null, new RMContainerTokenSecretManager(conf), new NMTokenSecretManagerInRM(conf), - new ClientToAMTokenSecretManagerInRM())); + new ClientToAMTokenSecretManagerInRM(), null)); CSQueue a = capacityScheduler.getQueue("a"); Assert.assertEquals(0.10, a.getAbsoluteCapacity(), DELTA); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java index b974528..db28dca 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java @@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; @@ -84,12 +85,13 @@ public EventHandler getEventHandler() { new ContainerAllocationExpirer(nullDispatcher); Configuration conf = new Configuration(); + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); RMContext rmContext = new RMContextImpl(nullDispatcher, cae, null, null, null, new AMRMTokenSecretManager(conf), new RMContainerTokenSecretManager(conf), new NMTokenSecretManagerInRM(conf), - new ClientToAMTokenSecretManagerInRM()); + new ClientToAMTokenSecretManagerInRM(), writer); return rmContext; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java index 005cf55..a7ad979 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java @@ -261,7 +261,7 @@ private ApplicationAttemptId createSchedulingRequest(int memory, int vcores, // This conditional is for testAclSubmitApplication where app is rejected // and no app is added. if (scheduler.applications.containsKey(id.getApplicationId())) { - scheduler.addApplicationAttempt(id, false); + scheduler.addApplicationAttempt(id); } List ask = new ArrayList(); ResourceRequest request = createResourceRequest(memory, vcores, ResourceRequest.ANY, @@ -590,7 +590,7 @@ public void testSimpleContainerReservation() throws Exception { // Make sure queue 2 is waiting with a reservation assertEquals(0, scheduler.getQueueManager().getQueue("queue2"). getResourceUsage().getMemory()); - assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory()); + assertEquals(1024, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory()); // Now another node checks in with capacity RMNode node2 = @@ -606,10 +606,10 @@ public void testSimpleContainerReservation() throws Exception { getResourceUsage().getMemory()); // The old reservation should still be there... - assertEquals(1024, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory()); + assertEquals(1024, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory()); // ... but it should disappear when we update the first node. scheduler.handle(updateEvent); - assertEquals(0, scheduler.getSchedulerApp(attId).getCurrentReservation().getMemory()); + assertEquals(0, scheduler.appAttempts.get(attId).getCurrentReservation().getMemory()); } @@ -630,7 +630,7 @@ public void testUserAsDefaultQueue() throws Exception { "user1"); scheduler.handle(appAddedEvent); AppAttemptAddedSchedulerEvent attempAddedEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId, false); + new AppAttemptAddedSchedulerEvent(appAttemptId); scheduler.handle(attempAddedEvent); assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true) .getRunnableAppSchedulables().size()); @@ -656,7 +656,7 @@ public void testNotUserAsDefaultQueue() throws Exception { "user2"); scheduler.handle(appAddedEvent); AppAttemptAddedSchedulerEvent attempAddedEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId, false); + new AppAttemptAddedSchedulerEvent(appAttemptId); scheduler.handle(attempAddedEvent); assertEquals(0, scheduler.getQueueManager().getLeafQueue("user1", true) .getRunnableAppSchedulables().size()); @@ -710,6 +710,7 @@ public void testQueuePlacementWithPolicy() throws Exception { scheduler.reinitialize(conf, resourceManager.getRMContext()); ApplicationAttemptId appId; + Map apps = scheduler.appAttempts; List rules = new ArrayList(); rules.add(new QueuePlacementRule.Specified().initialize(true, null)); @@ -722,17 +723,17 @@ public void testQueuePlacementWithPolicy() throws Exception { scheduler.getAllocationConfiguration().placementPolicy = new QueuePlacementPolicy(rules, queues, conf); appId = createSchedulingRequest(1024, "somequeue", "user1"); - assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName()); + assertEquals("root.somequeue", apps.get(appId).getQueueName()); appId = createSchedulingRequest(1024, "default", "user1"); - assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName()); + assertEquals("root.user1", apps.get(appId).getQueueName()); appId = createSchedulingRequest(1024, "default", "user3"); - assertEquals("root.user3group", scheduler.getSchedulerApp(appId).getQueueName()); + assertEquals("root.user3group", apps.get(appId).getQueueName()); appId = createSchedulingRequest(1024, "default", "user4"); - assertEquals("root.user4subgroup1", scheduler.getSchedulerApp(appId).getQueueName()); + assertEquals("root.user4subgroup1", apps.get(appId).getQueueName()); appId = createSchedulingRequest(1024, "default", "user5"); - assertEquals("root.user5subgroup2", scheduler.getSchedulerApp(appId).getQueueName()); + assertEquals("root.user5subgroup2", apps.get(appId).getQueueName()); appId = createSchedulingRequest(1024, "default", "otheruser"); - assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName()); + assertEquals("root.default", apps.get(appId).getQueueName()); // test without specified as first rule rules = new ArrayList(); @@ -742,11 +743,11 @@ public void testQueuePlacementWithPolicy() throws Exception { scheduler.getAllocationConfiguration().placementPolicy = new QueuePlacementPolicy(rules, queues, conf); appId = createSchedulingRequest(1024, "somequeue", "user1"); - assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName()); + assertEquals("root.user1", apps.get(appId).getQueueName()); appId = createSchedulingRequest(1024, "somequeue", "otheruser"); - assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName()); + assertEquals("root.somequeue", apps.get(appId).getQueueName()); appId = createSchedulingRequest(1024, "default", "otheruser"); - assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName()); + assertEquals("root.default", apps.get(appId).getQueueName()); } @Test @@ -801,13 +802,13 @@ public void testQueueDemandCalculation() throws Exception { ApplicationAttemptId id11 = createAppAttemptId(1, 1); scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1"); - scheduler.addApplicationAttempt(id11, false); + scheduler.addApplicationAttempt(id11); ApplicationAttemptId id21 = createAppAttemptId(2, 1); scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1"); - scheduler.addApplicationAttempt(id21, false); + scheduler.addApplicationAttempt(id21); ApplicationAttemptId id22 = createAppAttemptId(2, 2); scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1"); - scheduler.addApplicationAttempt(id22, false); + scheduler.addApplicationAttempt(id22); int minReqSize = FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB; @@ -853,7 +854,7 @@ public void testAppAdditionAndRemoval() throws Exception { "user1"); scheduler.handle(appAddedEvent); AppAttemptAddedSchedulerEvent attemptAddedEvent = - new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false); + new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1)); scheduler.handle(attemptAddedEvent); // Scheduler should have two queues (the default and the one created for user1) @@ -864,7 +865,7 @@ public void testAppAdditionAndRemoval() throws Exception { .getRunnableAppSchedulables().size()); AppAttemptRemovedSchedulerEvent appRemovedEvent1 = new AppAttemptRemovedSchedulerEvent( - createAppAttemptId(1, 1), RMAppAttemptState.FINISHED, false); + createAppAttemptId(1, 1), RMAppAttemptState.FINISHED); // Now remove app scheduler.handle(appRemovedEvent1); @@ -1137,12 +1138,12 @@ public void testChoiceOfPreemptedContainers() throws Exception { scheduler.handle(nodeUpdate3); } - assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app3).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app5).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app6).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app3).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app6).getLiveContainers().size()); // Now new requests arrive from queues C and D ApplicationAttemptId app7 = @@ -1165,16 +1166,16 @@ public void testChoiceOfPreemptedContainers() throws Exception { // Make sure it is lowest priority container. scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), Resources.createResource(2 * 1024)); - assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app2).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app5).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app2).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app5).getLiveContainers().size()); // First verify we are adding containers to preemption list for the application - assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app3).getLiveContainers(), - scheduler.getSchedulerApp(app3).getPreemptionContainers())); - assertTrue(!Collections.disjoint(scheduler.getSchedulerApp(app6).getLiveContainers(), - scheduler.getSchedulerApp(app6).getPreemptionContainers())); + assertTrue(!Collections.disjoint(scheduler.appAttempts.get(app3).getLiveContainers(), + scheduler.appAttempts.get(app3).getPreemptionContainers())); + assertTrue(!Collections.disjoint(scheduler.appAttempts.get(app6).getLiveContainers(), + scheduler.appAttempts.get(app6).getPreemptionContainers())); // Pretend 15 seconds have passed clock.tick(15); @@ -1184,8 +1185,8 @@ public void testChoiceOfPreemptedContainers() throws Exception { Resources.createResource(2 * 1024)); // At this point the containers should have been killed (since we are not simulating AM) - assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size()); - assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size()); // Trigger a kill by insisting we want containers back scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), @@ -1199,22 +1200,22 @@ public void testChoiceOfPreemptedContainers() throws Exception { scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), Resources.createResource(2 * 1024)); - assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size()); - assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size()); - assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size()); - assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size()); - assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size()); // Now A and B are below fair share, so preemption shouldn't do anything scheduler.preemptResources(scheduler.getQueueManager().getLeafQueues(), Resources.createResource(2 * 1024)); - assertEquals(1, scheduler.getSchedulerApp(app1).getLiveContainers().size()); - assertEquals(0, scheduler.getSchedulerApp(app2).getLiveContainers().size()); - assertEquals(0, scheduler.getSchedulerApp(app3).getLiveContainers().size()); - assertEquals(1, scheduler.getSchedulerApp(app4).getLiveContainers().size()); - assertEquals(0, scheduler.getSchedulerApp(app5).getLiveContainers().size()); - assertEquals(0, scheduler.getSchedulerApp(app6).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app1).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app2).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app3).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(app4).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app5).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(app6).getLiveContainers().size()); } @Test (timeout = 5000) @@ -1373,9 +1374,9 @@ public void testMultipleContainersWaitingForReservation() throws IOException { // One container should get reservation and the other should get nothing assertEquals(1024, - scheduler.getSchedulerApp(attId1).getCurrentReservation().getMemory()); + scheduler.appAttempts.get(attId1).getCurrentReservation().getMemory()); assertEquals(0, - scheduler.getSchedulerApp(attId2).getCurrentReservation().getMemory()); + scheduler.appAttempts.get(attId2).getCurrentReservation().getMemory()); } @Test (timeout = 5000) @@ -1410,7 +1411,7 @@ public void testUserMaxRunningApps() throws Exception { scheduler.handle(updateEvent); // App 1 should be running - assertEquals(1, scheduler.getSchedulerApp(attId1).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(attId1).getLiveContainers().size()); ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1", "user1", 1); @@ -1419,7 +1420,7 @@ public void testUserMaxRunningApps() throws Exception { scheduler.handle(updateEvent); // App 2 should not be running - assertEquals(0, scheduler.getSchedulerApp(attId2).getLiveContainers().size()); + assertEquals(0, scheduler.appAttempts.get(attId2).getLiveContainers().size()); // Request another container for app 1 createSchedulingRequestExistingApplication(1024, 1, attId1); @@ -1428,7 +1429,7 @@ public void testUserMaxRunningApps() throws Exception { scheduler.handle(updateEvent); // Request should be fulfilled - assertEquals(2, scheduler.getSchedulerApp(attId1).getLiveContainers().size()); + assertEquals(2, scheduler.appAttempts.get(attId1).getLiveContainers().size()); } @Test (timeout = 5000) @@ -1448,10 +1449,10 @@ public void testReservationWhileMultiplePriorities() throws IOException { NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); scheduler.handle(updateEvent); - FSSchedulerApp app = scheduler.getSchedulerApp(attId); + FSSchedulerApp app = scheduler.appAttempts.get(attId); assertEquals(1, app.getLiveContainers().size()); - ContainerId containerId = scheduler.getSchedulerApp(attId) + ContainerId containerId = scheduler.appAttempts.get(attId) .getLiveContainers().iterator().next().getContainerId(); // Cause reservation to be created @@ -1520,9 +1521,9 @@ public void testAclSubmitApplication() throws Exception { ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1", "norealuserhasthisname2", 1); - FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1); + FSSchedulerApp app1 = scheduler.appAttempts.get(attId1); assertNotNull("The application was not allowed", app1); - FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2); + FSSchedulerApp app2 = scheduler.appAttempts.get(attId2); assertNull("The application was allowed", app2); } @@ -1546,7 +1547,7 @@ public void testMultipleNodesSingleRackRequest() throws Exception { ApplicationAttemptId appId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); scheduler.addApplication(appId.getApplicationId(), "queue1", "user1"); - scheduler.addApplicationAttempt(appId, false); + scheduler.addApplicationAttempt(appId); // 1 request with 2 nodes on the same rack. another request with 1 node on // a different rack @@ -1565,14 +1566,14 @@ public void testMultipleNodesSingleRackRequest() throws Exception { NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1); scheduler.handle(updateEvent1); // should assign node local - assertEquals(1, scheduler.getSchedulerApp(appId).getLiveContainers().size()); + assertEquals(1, scheduler.appAttempts.get(appId).getLiveContainers().size()); // node 2 checks in scheduler.update(); NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2); scheduler.handle(updateEvent2); // should assign rack local - assertEquals(2, scheduler.getSchedulerApp(appId).getLiveContainers().size()); + assertEquals(2, scheduler.appAttempts.get(appId).getLiveContainers().size()); } @Test (timeout = 5000) @@ -1591,8 +1592,8 @@ public void testFifoWithinQueue() throws Exception { "user1", 2); ApplicationAttemptId attId2 = createSchedulingRequest(1024, "queue1", "user1", 2); - FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1); - FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2); + FSSchedulerApp app1 = scheduler.appAttempts.get(attId1); + FSSchedulerApp app2 = scheduler.appAttempts.get(attId2); FSLeafQueue queue1 = scheduler.getQueueManager().getLeafQueue("queue1", true); queue1.setPolicy(new FifoPolicy()); @@ -1632,7 +1633,7 @@ public void testMaxAssign() throws Exception { ApplicationAttemptId attId = createSchedulingRequest(1024, "root.default", "user", 8); - FSSchedulerApp app = scheduler.getSchedulerApp(attId); + FSSchedulerApp app = scheduler.appAttempts.get(attId); // set maxAssign to 2: only 2 containers should be allocated scheduler.maxAssign = 2; @@ -1694,10 +1695,10 @@ public void testAssignContainer() throws Exception { ApplicationAttemptId attId4 = createSchedulingRequest(1024, fifoQueue, user, 4); - FSSchedulerApp app1 = scheduler.getSchedulerApp(attId1); - FSSchedulerApp app2 = scheduler.getSchedulerApp(attId2); - FSSchedulerApp app3 = scheduler.getSchedulerApp(attId3); - FSSchedulerApp app4 = scheduler.getSchedulerApp(attId4); + FSSchedulerApp app1 = scheduler.appAttempts.get(attId1); + FSSchedulerApp app2 = scheduler.appAttempts.get(attId2); + FSSchedulerApp app3 = scheduler.appAttempts.get(attId3); + FSSchedulerApp app4 = scheduler.appAttempts.get(attId4); scheduler.getQueueManager().getLeafQueue(fifoQueue, true) .setPolicy(SchedulingPolicy.parse("fifo")); @@ -1812,7 +1813,7 @@ public void testReservationThatDoesntFit() throws IOException { NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); scheduler.handle(updateEvent); - FSSchedulerApp app = scheduler.getSchedulerApp(attId); + FSSchedulerApp app = scheduler.appAttempts.get(attId); assertEquals(0, app.getLiveContainers().size()); assertEquals(0, app.getReservedContainers().size()); @@ -1881,7 +1882,7 @@ public void testStrictLocality() throws IOException { NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2); // no matter how many heartbeats, node2 should never get a container - FSSchedulerApp app = scheduler.getSchedulerApp(attId1); + FSSchedulerApp app = scheduler.appAttempts.get(attId1); for (int i = 0; i < 10; i++) { scheduler.handle(node2UpdateEvent); assertEquals(0, app.getLiveContainers().size()); @@ -1920,7 +1921,7 @@ public void testCancelStrictLocality() throws IOException { NodeUpdateSchedulerEvent node2UpdateEvent = new NodeUpdateSchedulerEvent(node2); // no matter how many heartbeats, node2 should never get a container - FSSchedulerApp app = scheduler.getSchedulerApp(attId1); + FSSchedulerApp app = scheduler.appAttempts.get(attId1); for (int i = 0; i < 10; i++) { scheduler.handle(node2UpdateEvent); assertEquals(0, app.getLiveContainers().size()); @@ -1953,7 +1954,7 @@ public void testReservationsStrictLocality() throws IOException { ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1", "user1", 0); - FSSchedulerApp app = scheduler.getSchedulerApp(attId); + FSSchedulerApp app = scheduler.appAttempts.get(attId); ResourceRequest nodeRequest = createResourceRequest(1024, node2.getHostName(), 1, 2, true); ResourceRequest rackRequest = createResourceRequest(1024, "rack1", 1, 2, true); @@ -1993,7 +1994,7 @@ public void testNoMoreCpuOnNode() throws IOException { ApplicationAttemptId attId = createSchedulingRequest(1024, 1, "default", "user1", 2); - FSSchedulerApp app = scheduler.getSchedulerApp(attId); + FSSchedulerApp app = scheduler.appAttempts.get(attId); scheduler.update(); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); @@ -2013,10 +2014,10 @@ public void testBasicDRFAssignment() throws Exception { ApplicationAttemptId appAttId1 = createSchedulingRequest(2048, 1, "queue1", "user1", 2); - FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1); + FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1); ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 2, "queue1", "user1", 2); - FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2); + FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2); DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy(); drfPolicy.initialize(scheduler.getClusterCapacity()); @@ -2054,13 +2055,13 @@ public void testBasicDRFWithQueues() throws Exception { ApplicationAttemptId appAttId1 = createSchedulingRequest(3072, 1, "queue1", "user1", 2); - FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1); + FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1); ApplicationAttemptId appAttId2 = createSchedulingRequest(2048, 2, "queue1", "user1", 2); - FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2); + FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2); ApplicationAttemptId appAttId3 = createSchedulingRequest(1024, 2, "queue2", "user1", 2); - FSSchedulerApp app3 = scheduler.getSchedulerApp(appAttId3); + FSSchedulerApp app3 = scheduler.appAttempts.get(appAttId3); DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy(); drfPolicy.initialize(scheduler.getClusterCapacity()); @@ -2091,19 +2092,19 @@ public void testDRFHierarchicalQueues() throws Exception { ApplicationAttemptId appAttId1 = createSchedulingRequest(3074, 1, "queue1.subqueue1", "user1", 2); Thread.sleep(3); // so that start times will be different - FSSchedulerApp app1 = scheduler.getSchedulerApp(appAttId1); + FSSchedulerApp app1 = scheduler.appAttempts.get(appAttId1); ApplicationAttemptId appAttId2 = createSchedulingRequest(1024, 3, "queue1.subqueue1", "user1", 2); Thread.sleep(3); // so that start times will be different - FSSchedulerApp app2 = scheduler.getSchedulerApp(appAttId2); + FSSchedulerApp app2 = scheduler.appAttempts.get(appAttId2); ApplicationAttemptId appAttId3 = createSchedulingRequest(2048, 2, "queue1.subqueue2", "user1", 2); Thread.sleep(3); // so that start times will be different - FSSchedulerApp app3 = scheduler.getSchedulerApp(appAttId3); + FSSchedulerApp app3 = scheduler.appAttempts.get(appAttId3); ApplicationAttemptId appAttId4 = createSchedulingRequest(1024, 2, "queue2", "user1", 2); Thread.sleep(3); // so that start times will be different - FSSchedulerApp app4 = scheduler.getSchedulerApp(appAttId4); + FSSchedulerApp app4 = scheduler.appAttempts.get(appAttId4); DominantResourceFairnessPolicy drfPolicy = new DominantResourceFairnessPolicy(); drfPolicy.initialize(scheduler.getClusterCapacity()); @@ -2183,7 +2184,7 @@ public void testHostPortNodeName() throws Exception { NodeUpdateSchedulerEvent(node2); // no matter how many heartbeats, node2 should never get a container - FSSchedulerApp app = scheduler.getSchedulerApp(attId1); + FSSchedulerApp app = scheduler.appAttempts.get(attId1); for (int i = 0; i < 10; i++) { scheduler.handle(node2UpdateEvent); assertEquals(0, app.getLiveContainers().size()); @@ -2194,8 +2195,16 @@ public void testHostPortNodeName() throws Exception { assertEquals(1, app.getLiveContainers().size()); } + @Test + public void testConcurrentAccessOnApplications() throws Exception { + FairScheduler fs = new FairScheduler(); + TestCapacityScheduler.verifyConcurrentAccessOnApplications( + fs.appAttempts, FSSchedulerApp.class, FSLeafQueue.class); + } + + private void verifyAppRunnable(ApplicationAttemptId attId, boolean runnable) { - FSSchedulerApp app = scheduler.getSchedulerApp(attId); + FSSchedulerApp app = scheduler.appAttempts.get(attId); FSLeafQueue queue = app.getQueue(); Collection runnableApps = queue.getRunnableAppSchedulables(); @@ -2251,7 +2260,7 @@ public void testUserAndQueueMaxRunningApps() throws Exception { // Remove app 1 and both app 2 and app 4 should becomes runnable in its place AppAttemptRemovedSchedulerEvent appRemovedEvent1 = - new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED, false); + new AppAttemptRemovedSchedulerEvent(attId1, RMAppAttemptState.FINISHED); scheduler.handle(appRemovedEvent1); verifyAppRunnable(attId2, true); verifyQueueNumRunnable("queue2", 1, 0); @@ -2315,7 +2324,7 @@ public void testMaxRunningAppsHierarchicalQueues() throws Exception { // Even though the app was removed from sub3, the app from sub2 gets to go // because it came in first AppAttemptRemovedSchedulerEvent appRemovedEvent1 = - new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED, false); + new AppAttemptRemovedSchedulerEvent(attId2, RMAppAttemptState.FINISHED); scheduler.handle(appRemovedEvent1); verifyAppRunnable(attId4, true); verifyQueueNumRunnable("queue1.sub2", 2, 0); @@ -2324,7 +2333,7 @@ public void testMaxRunningAppsHierarchicalQueues() throws Exception { // Now test removal of a non-runnable app AppAttemptRemovedSchedulerEvent appRemovedEvent2 = - new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED, true); + new AppAttemptRemovedSchedulerEvent(attId5, RMAppAttemptState.KILLED); scheduler.handle(appRemovedEvent2); assertEquals(0, scheduler.maxRunningEnforcer.usersNonRunnableApps .get("user1").size()); @@ -2332,7 +2341,7 @@ public void testMaxRunningAppsHierarchicalQueues() throws Exception { verifyQueueNumRunnable("queue1.sub3", 0, 0); // verify it doesn't become runnable when there would be space for it AppAttemptRemovedSchedulerEvent appRemovedEvent3 = - new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED, true); + new AppAttemptRemovedSchedulerEvent(attId4, RMAppAttemptState.FINISHED); scheduler.handle(appRemovedEvent3); verifyQueueNumRunnable("queue1.sub2", 1, 0); verifyQueueNumRunnable("queue1.sub3", 0, 0); @@ -2369,7 +2378,7 @@ public void testContinuousScheduling() throws Exception { ApplicationAttemptId appAttemptId = createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); fs.addApplication(appAttemptId.getApplicationId(), "queue11", "user11"); - fs.addApplicationAttempt(appAttemptId, false); + fs.addApplicationAttempt(appAttemptId); List ask = new ArrayList(); ResourceRequest request = createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true); @@ -2380,7 +2389,7 @@ public void testContinuousScheduling() throws Exception { // at least one pass Thread.sleep(fs.getConf().getContinuousSchedulingSleepMs() + 500); - FSSchedulerApp app = fs.getSchedulerApp(appAttemptId); + FSSchedulerApp app = fs.appAttempts.get(appAttemptId); // Wait until app gets resources. while (app.getCurrentConsumption().equals(Resources.none())) { } @@ -2468,7 +2477,7 @@ public void testBlacklistNodes() throws Exception { ApplicationAttemptId appAttemptId = createSchedulingRequest(GB, "root.default", "user", 1); - FSSchedulerApp app = scheduler.getSchedulerApp(appAttemptId); + FSSchedulerApp app = scheduler.appAttempts.get(appAttemptId); // Verify the blacklist can be updated independent of requesting containers scheduler.allocate(appAttemptId, Collections.emptyList(), @@ -2478,7 +2487,7 @@ public void testBlacklistNodes() throws Exception { scheduler.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), null, Collections.singletonList(host)); - assertFalse(scheduler.getSchedulerApp(appAttemptId).isBlacklisted(host)); + assertFalse(scheduler.appAttempts.get(appAttemptId).isBlacklisted(host)); List update = Arrays.asList( createResourceRequest(GB, node.getHostName(), 1, 0, true)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java index 2a8c981..592cc85 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java @@ -21,6 +21,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; + import java.io.IOException; import java.lang.reflect.Method; import java.util.ArrayList; @@ -56,6 +58,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.Task; +import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; @@ -141,8 +144,9 @@ public void testFifoSchedulerCapacityWhenNoNMs() { @Test(timeout=5000) public void testAppAttemptMetrics() throws Exception { AsyncDispatcher dispatcher = new InlineDispatcher(); + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); RMContext rmContext = new RMContextImpl(dispatcher, null, - null, null, null, null, null, null, null); + null, null, null, null, null, null, null, writer); FifoScheduler schedular = new FifoScheduler(); schedular.reinitialize(new Configuration(), rmContext); @@ -156,7 +160,7 @@ public void testAppAttemptMetrics() throws Exception { SchedulerEvent appEvent = new AppAddedSchedulerEvent(appId, "queue", "user"); schedular.handle(appEvent); SchedulerEvent attemptEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId, false); + new AppAttemptAddedSchedulerEvent(appAttemptId); schedular.handle(attemptEvent); appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2); @@ -166,7 +170,7 @@ public void testAppAttemptMetrics() throws Exception { "user"); schedular.handle(appEvent2); SchedulerEvent attemptEvent2 = - new AppAttemptAddedSchedulerEvent(appAttemptId, false); + new AppAttemptAddedSchedulerEvent(appAttemptId); schedular.handle(attemptEvent2); int afterAppsSubmitted = metrics.getAppsSubmitted(); @@ -183,8 +187,9 @@ public void testNodeLocalAssignment() throws Exception { NMTokenSecretManagerInRM nmTokenSecretManager = new NMTokenSecretManagerInRM(conf); nmTokenSecretManager.rollMasterKey(); + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null, - null, containerTokenSecretManager, nmTokenSecretManager, null); + null, containerTokenSecretManager, nmTokenSecretManager, null, writer); FifoScheduler scheduler = new FifoScheduler(); scheduler.reinitialize(new Configuration(), rmContext); @@ -203,7 +208,7 @@ public void testNodeLocalAssignment() throws Exception { "user1"); scheduler.handle(appEvent); AppAttemptAddedSchedulerEvent attemptEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId, false); + new AppAttemptAddedSchedulerEvent(appAttemptId); scheduler.handle(attemptEvent); int memory = 64; @@ -247,8 +252,9 @@ public void testUpdateResourceOnNode() throws Exception { NMTokenSecretManagerInRM nmTokenSecretManager = new NMTokenSecretManagerInRM(conf); nmTokenSecretManager.rollMasterKey(); + RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null, - null, containerTokenSecretManager, nmTokenSecretManager, null); + null, containerTokenSecretManager, nmTokenSecretManager, null, writer); FifoScheduler scheduler = new FifoScheduler(){ @SuppressWarnings("unused") @@ -293,7 +299,7 @@ public void testUpdateResourceOnNode() throws Exception { "user1"); scheduler.handle(appEvent); AppAttemptAddedSchedulerEvent attemptEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId, false); + new AppAttemptAddedSchedulerEvent(appAttemptId); scheduler.handle(attemptEvent); int memory = 1024; @@ -534,6 +540,13 @@ public void testFifoScheduler() throws Exception { LOG.info("--- END: testFifoScheduler ---"); } + @Test + public void testConcurrentAccessOnApplications() throws Exception { + FifoScheduler fs = new FifoScheduler(); + TestCapacityScheduler.verifyConcurrentAccessOnApplications( + fs.appAttempts, FiCaSchedulerApp.class, Queue.class); + } + @SuppressWarnings("resource") @Test public void testBlackListNodes() throws Exception { @@ -557,18 +570,18 @@ public void testBlackListNodes() throws Exception { "user"); fs.handle(appEvent); SchedulerEvent attemptEvent = - new AppAttemptAddedSchedulerEvent(appAttemptId, false); + new AppAttemptAddedSchedulerEvent(appAttemptId); fs.handle(attemptEvent); // Verify the blacklist can be updated independent of requesting containers fs.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), Collections.singletonList(host), null); - Assert.assertTrue(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host)); + Assert.assertTrue(fs.getApplication(appAttemptId).isBlacklisted(host)); fs.allocate(appAttemptId, Collections.emptyList(), Collections.emptyList(), null, Collections.singletonList(host)); - Assert.assertFalse(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host)); + Assert.assertFalse(fs.getApplication(appAttemptId).isBlacklisted(host)); rm.stop(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java index 74dc95a..2c2aae6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java @@ -163,7 +163,7 @@ public static RMContext mockRMContext(int numApps, int racks, int numNodes, deactivatedNodesMap.put(node.getHostName(), node); } return new RMContextImpl(null, null, null, null, - null, null, null, null, null) { + null, null, null, null, null, null) { @Override public ConcurrentMap getRMApps() { return applicationsMaps; @@ -206,7 +206,7 @@ public static CapacityScheduler mockCapacityScheduler() throws IOException { cs.reinitialize(conf, new RMContextImpl(null, null, null, null, null, null, new RMContainerTokenSecretManager(conf), new NMTokenSecretManagerInRM(conf), - new ClientToAMTokenSecretManagerInRM())); + new ClientToAMTokenSecretManagerInRM(), null)); return cs; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java index 4f1208c..1ea2b0c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java @@ -29,7 +29,6 @@ import javax.xml.parsers.DocumentBuilderFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.ha.HAServiceProtocol; import org.apache.hadoop.service.Service.STATE; import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.yarn.api.records.QueueState; @@ -268,7 +267,6 @@ public void verifyClusterInfoXML(String xml) throws JSONException, Exception { verifyClusterGeneric(WebServicesTestUtils.getXmlLong(element, "id"), WebServicesTestUtils.getXmlLong(element, "startedOn"), WebServicesTestUtils.getXmlString(element, "state"), - WebServicesTestUtils.getXmlString(element, "haState"), WebServicesTestUtils.getXmlString(element, "hadoopVersionBuiltOn"), WebServicesTestUtils.getXmlString(element, "hadoopBuildVersion"), WebServicesTestUtils.getXmlString(element, "hadoopVersion"), @@ -284,10 +282,9 @@ public void verifyClusterInfo(JSONObject json) throws JSONException, Exception { assertEquals("incorrect number of elements", 1, json.length()); JSONObject info = json.getJSONObject("clusterInfo"); - assertEquals("incorrect number of elements", 10, info.length()); + assertEquals("incorrect number of elements", 9, info.length()); verifyClusterGeneric(info.getLong("id"), info.getLong("startedOn"), - info.getString("state"), info.getString("haState"), - info.getString("hadoopVersionBuiltOn"), + info.getString("state"), info.getString("hadoopVersionBuiltOn"), info.getString("hadoopBuildVersion"), info.getString("hadoopVersion"), info.getString("resourceManagerVersionBuiltOn"), info.getString("resourceManagerBuildVersion"), @@ -296,10 +293,9 @@ public void verifyClusterInfo(JSONObject json) throws JSONException, } public void verifyClusterGeneric(long clusterid, long startedon, - String state, String haState, String hadoopVersionBuiltOn, - String hadoopBuildVersion, String hadoopVersion, - String resourceManagerVersionBuiltOn, String resourceManagerBuildVersion, - String resourceManagerVersion) { + String state, String hadoopVersionBuiltOn, String hadoopBuildVersion, + String hadoopVersion, String resourceManagerVersionBuiltOn, + String resourceManagerBuildVersion, String resourceManagerVersion) { assertEquals("clusterId doesn't match: ", ResourceManager.getClusterTimeStamp(), clusterid); @@ -307,8 +303,6 @@ public void verifyClusterGeneric(long clusterid, long startedon, ResourceManager.getClusterTimeStamp(), startedon); assertTrue("stated doesn't match: " + state, state.matches(STATE.INITED.toString())); - assertTrue("HA state doesn't match: " + haState, - haState.matches("INITIALIZING")); WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn", VersionInfo.getDate(), hadoopVersionBuiltOn); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java index ab6bb35..58170ef 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java @@ -1390,7 +1390,7 @@ public void testAppAttempts() throws JSONException, Exception { @Test public void testMultipleAppAttempts() throws JSONException, Exception { rm.start(); - MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 8192); + MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); RMApp app1 = rm.submitApp(CONTAINER_MB, "testwordcount", "user1"); amNodeManager.nodeHeartbeat(true); rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(), @@ -1403,13 +1403,11 @@ public void testMultipleAppAttempts() throws JSONException, Exception { while (--retriesLeft > 0) { RMAppEvent event = new RMAppFailedAttemptEvent(app1.getApplicationId(), - RMAppEventType.ATTEMPT_FAILED, "", false); + RMAppEventType.ATTEMPT_FAILED, ""); app1.handle(event); rm.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED); amNodeManager.nodeHeartbeat(true); } - // kick the scheduler to allocate the am container. - amNodeManager.nodeHeartbeat(true); rm.waitForState(app1.getCurrentAppAttempt().getAppAttemptId(), RMAppAttemptState.ALLOCATED); assertEquals("incorrect number of attempts", maxAppAttempts, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml index 6a2a73e..2bd682b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml @@ -47,5 +47,6 @@ hadoop-yarn-server-web-proxy hadoop-yarn-server-resourcemanager hadoop-yarn-server-tests + hadoop-yarn-server-applicationhistoryservice diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm index 7390291..8f7300a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerRest.apt.vm @@ -62,8 +62,6 @@ ResourceManager REST API's. *---------------+--------------+-------------------------------+ | state | string | The ResourceManager state - valid values are: NOTINITED, INITED, STARTED, STOPPED| *---------------+--------------+-------------------------------+ -| haState | string | The ResourceManager HA state - valid values are: INITIALIZING, ACTIVE, STANDBY, STOPPED| -*---------------+--------------+-------------------------------+ | resourceManagerVersion | string | Version of the ResourceManager | *---------------+--------------+-------------------------------+ | resourceManagerBuildVersion | string | ResourceManager build string with build version, user, and checksum |