diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
index cbd941b6b9d90..7a03240e80d9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
@@ -77,7 +77,7 @@ public class ClientContext {
/**
* Caches short-circuit file descriptors, mmap regions.
*/
- private final ShortCircuitCache shortCircuitCache;
+ private final ShortCircuitCache[] shortCircuitCache;
/**
* Caches TCP and UNIX domain sockets for reuse.
@@ -132,13 +132,23 @@ public class ClientContext {
*/
private DeadNodeDetector deadNodeDetector = null;
+ /**
+ * ShortCircuitCache array size.
+ */
+ private final int clientShortCircuitNum;
+
private ClientContext(String name, DfsClientConf conf,
Configuration config) {
final ShortCircuitConf scConf = conf.getShortCircuitConf();
this.name = name;
this.confString = scConf.confAsString();
- this.shortCircuitCache = ShortCircuitCache.fromConf(scConf);
+ this.clientShortCircuitNum = conf.getClientShortCircuitNum();
+ this.shortCircuitCache = new ShortCircuitCache[this.clientShortCircuitNum];
+ for (int i = 0; i < this.clientShortCircuitNum; i++) {
+ this.shortCircuitCache[i] = ShortCircuitCache.fromConf(scConf);
+ }
+
this.peerCache = new PeerCache(scConf.getSocketCacheCapacity(),
scConf.getSocketCacheExpiry());
this.keyProviderCache = new KeyProviderCache(
@@ -228,7 +238,11 @@ public String getConfString() {
}
public ShortCircuitCache getShortCircuitCache() {
- return shortCircuitCache;
+ return shortCircuitCache[0];
+ }
+
+ public ShortCircuitCache getShortCircuitCache(long idx) {
+ return shortCircuitCache[(int) (idx % clientShortCircuitNum)];
}
public PeerCache getPeerCache() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index ab3f6f27211de..e8b540286c280 100755
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -144,6 +144,8 @@ public interface HdfsClientConfigKeys {
"dfs.short.circuit.shared.memory.watcher.interrupt.check.ms";
int DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT =
60000;
+ String DFS_CLIENT_SHORT_CIRCUIT_NUM = "dfs.client.short.circuit.num";
+ int DFS_CLIENT_SHORT_CIRCUIT_NUM_DEFAULT = 1;
String DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY =
"dfs.client.slow.io.warning.threshold.ms";
long DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT = 30000;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java
index a3b611c1ca7a2..028d6296e9ec6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java
@@ -476,7 +476,8 @@ private BlockReader getBlockReaderLocal() throws IOException {
"giving up on BlockReaderLocal.", this, pathInfo);
return null;
}
- ShortCircuitCache cache = clientContext.getShortCircuitCache();
+ ShortCircuitCache cache =
+ clientContext.getShortCircuitCache(block.getBlockId());
ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
block.getBlockPoolId());
ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
@@ -527,7 +528,8 @@ public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
if (curPeer.fromCache) remainingCacheTries--;
DomainPeer peer = (DomainPeer)curPeer.peer;
Slot slot = null;
- ShortCircuitCache cache = clientContext.getShortCircuitCache();
+ ShortCircuitCache cache =
+ clientContext.getShortCircuitCache(block.getBlockId());
try {
MutableBoolean usedPeer = new MutableBoolean(false);
slot = cache.allocShmSlot(datanode, peer, usedPeer,
@@ -582,7 +584,8 @@ public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
*/
private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer,
Slot slot) throws IOException {
- ShortCircuitCache cache = clientContext.getShortCircuitCache();
+ ShortCircuitCache cache =
+ clientContext.getShortCircuitCache(block.getBlockId());
final DataOutputStream out =
new DataOutputStream(new BufferedOutputStream(peer.getOutputStream(), SMALL_BUFFER_SIZE));
SlotId slotId = slot == null ? null : slot.getSlotId();
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
index 918fef7e50744..889e8000f4ae6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
@@ -142,6 +142,7 @@ public class DfsClientConf {
private final long refreshReadBlockLocationsMS;
private final ShortCircuitConf shortCircuitConf;
+ private final int clientShortCircuitNum;
private final long hedgedReadThresholdMillis;
private final int hedgedReadThreadpoolSize;
@@ -296,6 +297,18 @@ public DfsClientConf(Configuration conf) {
leaseHardLimitPeriod =
conf.getLong(HdfsClientConfigKeys.DFS_LEASE_HARDLIMIT_KEY,
HdfsClientConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT) * 1000;
+
+
+ clientShortCircuitNum = conf.getInt(
+ HdfsClientConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_NUM,
+ HdfsClientConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_NUM_DEFAULT);
+ Preconditions.checkArgument(clientShortCircuitNum >= 1,
+ HdfsClientConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_NUM +
+ "can't be less then 1.");
+ Preconditions.checkArgument(clientShortCircuitNum <= 5,
+ HdfsClientConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_NUM +
+ "can't be more then 5.");
+
}
@SuppressWarnings("unchecked")
@@ -601,6 +614,13 @@ public long getSlowIoWarningThresholdMs() {
return slowIoWarningThresholdMs;
}
+ /*
+ * @return the clientShortCircuitNum
+ */
+ public int getClientShortCircuitNum() {
+ return clientShortCircuitNum;
+ }
+
/**
* @return the hedgedReadThresholdMillis
*/
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 5c48b1870071c..79e13609fc0d2 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -4178,6 +4178,16 @@
+
+ dfs.client.short.circuit.num
+ 1
+
+ Number of short-circuit caches. This setting should
+ be in the range 1 - 5. Lower values will result in lower CPU consumption; higher
+ values may speed up massive parallel reading files.
+
+
+
dfs.client.read.striped.threadpool.size18
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
index 90b4f11a66a31..19bc71111e868 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
@@ -358,7 +358,7 @@ public void testZeroCopyMmapCache() throws Exception {
fsIn.close();
fsIn = fs.open(TEST_PATH);
final ShortCircuitCache cache = ClientContext.get(
- CONTEXT, conf).getShortCircuitCache();
+ CONTEXT, conf).getShortCircuitCache(0);
cache.accept(new CountingVisitor(0, 5, 5, 0));
results[0] = fsIn.read(null, BLOCK_SIZE,
EnumSet.of(ReadOption.SKIP_CHECKSUMS));
@@ -654,12 +654,12 @@ public void testZeroCopyReadOfCachedData() throws Exception {
BLOCK_SIZE), byteBufferToArray(result2));
fsIn2.releaseBuffer(result2);
fsIn2.close();
-
+
// check that the replica is anchored
final ExtendedBlock firstBlock =
DFSTestUtil.getFirstBlock(fs, TEST_PATH);
final ShortCircuitCache cache = ClientContext.get(
- CONTEXT, conf).getShortCircuitCache();
+ CONTEXT, conf).getShortCircuitCache(0);
waitForReplicaAnchorStatus(cache, firstBlock, true, true, 1);
// Uncache the replica
fs.removeCacheDirective(directiveId);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderFactory.java
index 6b04b14f49a77..84424494462e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderFactory.java
@@ -389,7 +389,7 @@ private void testShortCircuitCacheUnbufferWithDisableInterval(
try (FSDataInputStream in = dfs.open(testFile)) {
Assert.assertEquals(0,
- dfs.getClient().getClientContext().getShortCircuitCache()
+ dfs.getClient().getClientContext().getShortCircuitCache(0)
.getReplicaInfoMapSize());
final byte[] buf = new byte[testFileLen];
@@ -398,12 +398,12 @@ private void testShortCircuitCacheUnbufferWithDisableInterval(
// Set cache size to 0 so the replica marked evictable by unbuffer
// will be purged immediately.
- dfs.getClient().getClientContext().getShortCircuitCache()
+ dfs.getClient().getClientContext().getShortCircuitCache(0)
.setMaxTotalSize(0);
LOG.info("Unbuffering");
in.unbuffer();
Assert.assertEquals(0,
- dfs.getClient().getClientContext().getShortCircuitCache()
+ dfs.getClient().getClientContext().getShortCircuitCache(0)
.getReplicaInfoMapSize());
DFSTestUtil.appendFile(dfs, testFile, "append more data");
@@ -432,7 +432,7 @@ private void validateReadResult(final DistributedFileSystem dfs,
final int expectedScrRepMapSize) {
Assert.assertThat(expected, CoreMatchers.is(actual));
Assert.assertEquals(expectedScrRepMapSize,
- dfs.getClient().getClientContext().getShortCircuitCache()
+ dfs.getClient().getClientContext().getShortCircuitCache(0)
.getReplicaInfoMapSize());
}
@@ -467,7 +467,7 @@ public void testShortCircuitReadFromServerWithoutShm() throws Exception {
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
Assert.assertTrue(Arrays.equals(contents, expected));
final ShortCircuitCache cache =
- fs.getClient().getClientContext().getShortCircuitCache();
+ fs.getClient().getClientContext().getShortCircuitCache(0);
final DatanodeInfo datanode = new DatanodeInfoBuilder()
.setNodeID(cluster.getDataNodes().get(0).getDatanodeId())
.build();
@@ -516,7 +516,7 @@ public void testShortCircuitReadFromClientWithoutShm() throws Exception {
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
Assert.assertTrue(Arrays.equals(contents, expected));
final ShortCircuitCache cache =
- fs.getClient().getClientContext().getShortCircuitCache();
+ fs.getClient().getClientContext().getShortCircuitCache(0);
Assert.assertEquals(null, cache.getDfsClientShmManager());
cluster.shutdown();
sockDir.close();
@@ -548,7 +548,7 @@ public void testShortCircuitCacheShutdown() throws Exception {
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
Assert.assertTrue(Arrays.equals(contents, expected));
final ShortCircuitCache cache =
- fs.getClient().getClientContext().getShortCircuitCache();
+ fs.getClient().getClientContext().getShortCircuitCache(0);
cache.close();
Assert.assertTrue(cache.getDfsClientShmManager().
getDomainSocketWatcher().isClosed());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java
index 95fb67a1a4e19..c9cd14e5c95b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java
@@ -116,7 +116,7 @@ private static void readFully(BlockReaderLocal reader,
}
private static class BlockReaderLocalTest {
- final static int TEST_LENGTH = 12345;
+ final static int TEST_LENGTH = 1234567;
final static int BYTES_PER_CHECKSUM = 512;
public void setConfiguration(HdfsConfiguration conf) {
@@ -130,10 +130,14 @@ public void doTest(BlockReaderLocal reader, byte original[])
throws IOException {
// default: no-op
}
- }
+ public void doTest(BlockReaderLocal reader, byte[] original, int shift)
+ throws IOException {
+ // default: no-op
+ } }
public void runBlockReaderLocalTest(BlockReaderLocalTest test,
- boolean checksum, long readahead) throws IOException {
+ boolean checksum, long readahead, int shortCircuitCachesNum)
+ throws IOException {
Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
MiniDFSCluster cluster = null;
HdfsConfiguration conf = new HdfsConfiguration();
@@ -143,10 +147,13 @@ public void runBlockReaderLocalTest(BlockReaderLocalTest test,
BlockReaderLocalTest.BYTES_PER_CHECKSUM);
conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD, readahead);
+ conf.setInt(HdfsClientConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_NUM,
+ shortCircuitCachesNum);
test.setConfiguration(conf);
FileInputStream dataIn = null, metaIn = null;
final Path TEST_PATH = new Path("/a");
final long RANDOM_SEED = 4567L;
+ final int blockSize = 10 * 1024;
BlockReaderLocal blockReaderLocal = null;
FSDataInputStream fsIn = null;
byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
@@ -158,8 +165,8 @@ public void runBlockReaderLocalTest(BlockReaderLocalTest test,
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
cluster.waitActive();
fs = cluster.getFileSystem();
- DFSTestUtil.createFile(fs, TEST_PATH,
- BlockReaderLocalTest.TEST_LENGTH, (short)1, RANDOM_SEED);
+ DFSTestUtil.createFile(fs, TEST_PATH, 1024,
+ BlockReaderLocalTest.TEST_LENGTH, blockSize, (short)1, RANDOM_SEED);
try {
DFSTestUtil.waitReplication(fs, TEST_PATH, (short)1);
} catch (InterruptedException e) {
@@ -174,47 +181,52 @@ public void runBlockReaderLocalTest(BlockReaderLocalTest test,
BlockReaderLocalTest.TEST_LENGTH);
fsIn.close();
fsIn = null;
- ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_PATH);
- File dataFile = cluster.getBlockFile(0, block);
- File metaFile = cluster.getBlockMetadataFile(0, block);
-
- ShortCircuitCache shortCircuitCache =
- ClientContext.getFromConf(conf).getShortCircuitCache();
+ for (int i = 0; i < shortCircuitCachesNum; i++) {
+ ExtendedBlock block = DFSTestUtil.getAllBlocks(
+ fs, TEST_PATH).get(i).getBlock();
+ File dataFile = cluster.getBlockFile(0, block);
+ File metaFile = cluster.getBlockMetadataFile(0, block);
+
+ ShortCircuitCache shortCircuitCache =
+ ClientContext.getFromConf(conf).getShortCircuitCache(
+ block.getBlockId());
+ test.setup(dataFile, checksum);
+ FileInputStream[] streams = {
+ new FileInputStream(dataFile),
+ new FileInputStream(metaFile)
+ };
+ dataIn = streams[0];
+ metaIn = streams[1];
+ ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
+ block.getBlockPoolId());
+ raf = new RandomAccessFile(
+ new File(sockDir.getDir().getAbsolutePath(),
+ UUID.randomUUID().toString()), "rw");
+ raf.setLength(8192);
+ FileInputStream shmStream = new FileInputStream(raf.getFD());
+ shm = new ShortCircuitShm(ShmId.createRandom(), shmStream);
+ ShortCircuitReplica replica =
+ new ShortCircuitReplica(key, dataIn, metaIn, shortCircuitCache,
+ Time.now(), shm.allocAndRegisterSlot(
+ ExtendedBlockId.fromExtendedBlock(block)));
+ blockReaderLocal = new BlockReaderLocal.Builder(
+ new DfsClientConf.ShortCircuitConf(conf)).
+ setFilename(TEST_PATH.getName()).
+ setBlock(block).
+ setShortCircuitReplica(replica).
+ setCachingStrategy(new CachingStrategy(false, readahead)).
+ setVerifyChecksum(checksum).
+ build();
+ dataIn = null;
+ metaIn = null;
+ test.doTest(blockReaderLocal, original, i * blockSize);
+ // BlockReaderLocal should not alter the file position.
+ Assert.assertEquals(0, streams[0].getChannel().position());
+ Assert.assertEquals(0, streams[1].getChannel().position());
+ }
cluster.shutdown();
cluster = null;
- test.setup(dataFile, checksum);
- FileInputStream streams[] = {
- new FileInputStream(dataFile),
- new FileInputStream(metaFile)
- };
- dataIn = streams[0];
- metaIn = streams[1];
- ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
- block.getBlockPoolId());
- raf = new RandomAccessFile(
- new File(sockDir.getDir().getAbsolutePath(),
- UUID.randomUUID().toString()), "rw");
- raf.setLength(8192);
- FileInputStream shmStream = new FileInputStream(raf.getFD());
- shm = new ShortCircuitShm(ShmId.createRandom(), shmStream);
- ShortCircuitReplica replica =
- new ShortCircuitReplica(key, dataIn, metaIn, shortCircuitCache,
- Time.now(), shm.allocAndRegisterSlot(
- ExtendedBlockId.fromExtendedBlock(block)));
- blockReaderLocal = new BlockReaderLocal.Builder(
- new DfsClientConf.ShortCircuitConf(conf)).
- setFilename(TEST_PATH.getName()).
- setBlock(block).
- setShortCircuitReplica(replica).
- setCachingStrategy(new CachingStrategy(false, readahead)).
- setVerifyChecksum(checksum).
- build();
- dataIn = null;
- metaIn = null;
- test.doTest(blockReaderLocal, original);
- // BlockReaderLocal should not alter the file position.
- Assert.assertEquals(0, streams[0].getChannel().position());
- Assert.assertEquals(0, streams[1].getChannel().position());
+
} finally {
if (fsIn != null) fsIn.close();
if (fs != null) fs.close();
@@ -227,14 +239,17 @@ public void runBlockReaderLocalTest(BlockReaderLocalTest test,
}
}
+
private static class TestBlockReaderLocalImmediateClose
extends BlockReaderLocalTest {
}
@Test
public void testBlockReaderLocalImmediateClose() throws IOException {
- runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), true, 0);
- runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), false, 0);
+ runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(),
+ true, 0, 1);
+ runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(),
+ false, 0, 1);
}
private static class TestBlockReaderSimpleReads
@@ -242,7 +257,7 @@ private static class TestBlockReaderSimpleReads
@Override
public void doTest(BlockReaderLocal reader, byte original[])
throws IOException {
- byte buf[] = new byte[TEST_LENGTH];
+ byte[] buf = new byte[TEST_LENGTH];
reader.readFully(buf, 0, 512);
assertArrayRegionsEqual(original, 0, buf, 0, 512);
reader.readFully(buf, 512, 512);
@@ -260,30 +275,30 @@ public void doTest(BlockReaderLocal reader, byte original[])
@Test
public void testBlockReaderSimpleReads() throws IOException {
runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true,
- HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderSimpleReadsShortReadahead() throws IOException {
runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true,
- BlockReaderLocalTest.BYTES_PER_CHECKSUM - 1);
+ BlockReaderLocalTest.BYTES_PER_CHECKSUM - 1, 1);
}
@Test
public void testBlockReaderSimpleReadsNoChecksum() throws IOException {
runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false,
- HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderSimpleReadsNoReadahead() throws IOException {
- runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true, 0);
+ runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true, 0, 1);
}
@Test
public void testBlockReaderSimpleReadsNoChecksumNoReadahead()
throws IOException {
- runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false, 0);
+ runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false, 0, 1);
}
private static class TestBlockReaderLocalArrayReads2
@@ -291,7 +306,7 @@ private static class TestBlockReaderLocalArrayReads2
@Override
public void doTest(BlockReaderLocal reader, byte original[])
throws IOException {
- byte buf[] = new byte[TEST_LENGTH];
+ byte[] buf = new byte[TEST_LENGTH];
reader.readFully(buf, 0, 10);
assertArrayRegionsEqual(original, 0, buf, 0, 10);
reader.readFully(buf, 10, 100);
@@ -310,26 +325,26 @@ public void doTest(BlockReaderLocal reader, byte original[])
@Test
public void testBlockReaderLocalArrayReads2() throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
- true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalArrayReads2NoChecksum()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
- false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalArrayReads2NoReadahead()
throws IOException {
- runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), true, 0);
+ runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), true, 0, 1);
}
@Test
public void testBlockReaderLocalArrayReads2NoChecksumNoReadahead()
throws IOException {
- runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), false, 0);
+ runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), false, 0, 1);
}
private static class TestBlockReaderLocalByteBufferReads
@@ -354,7 +369,7 @@ public void doTest(BlockReaderLocal reader, byte original[])
public void testBlockReaderLocalByteBufferReads()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
- true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
@@ -362,21 +377,21 @@ public void testBlockReaderLocalByteBufferReadsNoChecksum()
throws IOException {
runBlockReaderLocalTest(
new TestBlockReaderLocalByteBufferReads(),
- false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalByteBufferReadsNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
- true, 0);
+ true, 0, 1);
}
@Test
public void testBlockReaderLocalByteBufferReadsNoChecksumNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferReads(),
- false, 0);
+ false, 0, 1);
}
/**
@@ -425,7 +440,7 @@ public void doTest(BlockReaderLocal reader, byte original[])
public void testBlockReaderLocalByteBufferFastLaneReads()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferFastLaneReads(),
- true, 2 * BlockReaderLocalTest.BYTES_PER_CHECKSUM);
+ true, 2 * BlockReaderLocalTest.BYTES_PER_CHECKSUM, 1);
}
@Test
@@ -433,21 +448,21 @@ public void testBlockReaderLocalByteBufferFastLaneReadsNoChecksum()
throws IOException {
runBlockReaderLocalTest(
new TestBlockReaderLocalByteBufferFastLaneReads(),
- false, 2 * BlockReaderLocalTest.BYTES_PER_CHECKSUM);
+ false, 2 * BlockReaderLocalTest.BYTES_PER_CHECKSUM, 1);
}
@Test
public void testBlockReaderLocalByteBufferFastLaneReadsNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferFastLaneReads(),
- true, 0);
+ true, 0, 1);
}
@Test
public void testBlockReaderLocalByteBufferFastLaneReadsNoChecksumNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalByteBufferFastLaneReads(),
- false, 0);
+ false, 0, 1);
}
private static class TestBlockReaderLocalReadCorruptStart
@@ -468,7 +483,7 @@ public void setup(File blockFile, boolean usingChecksums)
public void doTest(BlockReaderLocal reader, byte original[])
throws IOException {
- byte buf[] = new byte[TEST_LENGTH];
+ byte[] buf = new byte[TEST_LENGTH];
if (usingChecksums) {
try {
reader.readFully(buf, 0, 10);
@@ -486,7 +501,7 @@ public void doTest(BlockReaderLocal reader, byte original[])
public void testBlockReaderLocalReadCorruptStart()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true,
- HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
private static class TestBlockReaderLocalReadCorrupt
@@ -508,7 +523,7 @@ public void setup(File blockFile, boolean usingChecksums)
public void doTest(BlockReaderLocal reader, byte original[])
throws IOException {
- byte buf[] = new byte[TEST_LENGTH];
+ byte[] buf = new byte[TEST_LENGTH];
try {
reader.readFully(buf, 0, 10);
assertArrayRegionsEqual(original, 0, buf, 0, 10);
@@ -537,26 +552,26 @@ public void doTest(BlockReaderLocal reader, byte original[])
public void testBlockReaderLocalReadCorrupt()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true,
- HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalReadCorruptNoChecksum()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false,
- HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalReadCorruptNoReadahead()
throws IOException {
- runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true, 0);
+ runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), true, 0, 1);
}
@Test
public void testBlockReaderLocalReadCorruptNoChecksumNoReadahead()
throws IOException {
- runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false, 0);
+ runBlockReaderLocalTest(new TestBlockReaderLocalReadCorrupt(), false, 0, 1);
}
private static class TestBlockReaderLocalWithMlockChanges
@@ -589,28 +604,28 @@ public void doTest(BlockReaderLocal reader, byte original[])
public void testBlockReaderLocalWithMlockChanges()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
- true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalWithMlockChangesNoChecksum()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
- false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalWithMlockChangesNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
- true, 0);
+ true, 0, 1);
}
@Test
public void testBlockReaderLocalWithMlockChangesNoChecksumNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalWithMlockChanges(),
- false, 0);
+ false, 0, 1);
}
private static class TestBlockReaderLocalOnFileWithoutChecksum
@@ -662,56 +677,56 @@ public void doTest(BlockReaderLocal reader, byte original[])
public void testBlockReaderLocalOnFileWithoutChecksum()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
- true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalOnFileWithoutChecksumNoChecksum()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
- false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalOnFileWithoutChecksumNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
- true, 0);
+ true, 0, 1);
}
@Test
public void testBlockReaderLocalOnFileWithoutChecksumNoChecksumNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
- false, 0);
+ false, 0, 1);
}
@Test
public void testBlockReaderLocalReadZeroBytes()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
- true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalReadZeroBytesNoChecksum()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
- false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+ false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT, 1);
}
@Test
public void testBlockReaderLocalReadZeroBytesNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
- true, 0);
+ true, 0, 1);
}
@Test
public void testBlockReaderLocalReadZeroBytesNoChecksumNoReadahead()
throws IOException {
runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
- false, 0);
+ false, 0, 1);
}
@@ -845,4 +860,78 @@ public void testStatisticsForErasureCodingRead() throws IOException {
}
}
}
+
+ private static class TestBlockReaderFiveShortCircutCachesReads
+ extends BlockReaderLocalTest {
+ @Override
+ public void doTest(BlockReaderLocal reader, byte[] original, int shift)
+ throws IOException {
+ byte[] buf = new byte[TEST_LENGTH];
+ reader.readFully(buf, 0, 512);
+ assertArrayRegionsEqual(original, shift, buf, 0, 512);
+ reader.readFully(buf, 512, 512);
+ assertArrayRegionsEqual(original, 512 + shift, buf, 512, 512);
+ reader.readFully(buf, 1024, 513);
+ assertArrayRegionsEqual(original, 1024 + shift, buf, 1024, 513);
+ reader.readFully(buf, 1537, 514);
+ assertArrayRegionsEqual(original, 1537 + shift, buf, 1537, 514);
+ // Readahead is always at least the size of one chunk in this test.
+ Assert.assertTrue(reader.getMaxReadaheadLength() >=
+ BlockReaderLocalTest.BYTES_PER_CHECKSUM);
+ }
+ }
+
+ @Test
+ public void testBlockReaderFiveShortCircutCachesReads() throws IOException {
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT,
+ 5);
+ }
+
+ @Test
+ public void testBlockReaderFiveShortCircutCachesReadsShortReadahead()
+ throws IOException {
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
+ true, BlockReaderLocalTest.BYTES_PER_CHECKSUM - 1,
+ 5);
+ }
+
+ @Test
+ public void testBlockReaderFiveShortCircutCachesReadsNoChecksum()
+ throws IOException {
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
+ false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT,
+ 5);
+ }
+
+ @Test
+ public void testBlockReaderFiveShortCircutCachesReadsNoReadahead()
+ throws IOException {
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
+ true, 0, 5);
+ }
+
+ @Test
+ public void testBlockReaderFiveShortCircutCachesReadsNoChecksumNoReadahead()
+ throws IOException {
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
+ false, 0, 5);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testBlockReaderShortCircutCachesOutOfRangeBelow()
+ throws IOException {
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT,
+ 0);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testBlockReaderShortCircutCachesOutOfRangeAbove()
+ throws IOException {
+ runBlockReaderLocalTest(new TestBlockReaderFiveShortCircutCachesReads(),
+ true, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT,
+ 555);
+ }
+
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index b2da6a2fca386..53cac2adee350 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -431,7 +431,7 @@ public void testAllocShm() throws Exception {
cluster.waitActive();
DistributedFileSystem fs = cluster.getFileSystem();
final ShortCircuitCache cache =
- fs.getClient().getClientContext().getShortCircuitCache();
+ fs.getClient().getClientContext().getShortCircuitCache(0);
cache.getDfsClientShmManager().visit(new Visitor() {
@Override
public void visit(HashMap info)
@@ -501,7 +501,7 @@ public void testShmBasedStaleness() throws Exception {
cluster.waitActive();
DistributedFileSystem fs = cluster.getFileSystem();
final ShortCircuitCache cache =
- fs.getClient().getClientContext().getShortCircuitCache();
+ fs.getClient().getClientContext().getShortCircuitCache(0);
String TEST_FILE = "/test_file";
final int TEST_FILE_LEN = 8193;
final int SEED = 0xFADED;
@@ -565,7 +565,7 @@ public void testUnlinkingReplicasInFileDescriptorCache() throws Exception {
cluster.waitActive();
DistributedFileSystem fs = cluster.getFileSystem();
final ShortCircuitCache cache =
- fs.getClient().getClientContext().getShortCircuitCache();
+ fs.getClient().getClientContext().getShortCircuitCache(0);
cache.getDfsClientShmManager().visit(new Visitor() {
@Override
public void visit(HashMap info)
@@ -877,19 +877,20 @@ public void testRequestFileDescriptorsWhenULimit() throws Exception {
return peerCache;
});
- Mockito.when(clientContext.getShortCircuitCache()).thenAnswer(
+ Mockito.when(clientContext.getShortCircuitCache(
+ blk.getBlock().getBlockId())).thenAnswer(
(Answer) shortCircuitCacheCall -> {
- ShortCircuitCache cache = Mockito.mock(ShortCircuitCache.class);
- Mockito.when(cache.allocShmSlot(
+ ShortCircuitCache cache = Mockito.mock(ShortCircuitCache.class);
+ Mockito.when(cache.allocShmSlot(
Mockito.any(DatanodeInfo.class),
Mockito.any(DomainPeer.class),
Mockito.any(MutableBoolean.class),
Mockito.any(ExtendedBlockId.class),
Mockito.anyString()))
- .thenAnswer((Answer) call -> null);
+ .thenAnswer((Answer) call -> null);
- return cache;
- }
+ return cache;
+ }
);
DatanodeInfo[] nodes = blk.getLocations();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/README.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/README.md
deleted file mode 100644
index 6e6ca594536c6..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/README.md
+++ /dev/null
@@ -1,89 +0,0 @@
-
-
-# YARN UI
-
-The YARN UI is an Ember based web-app that provides visualization of the applications running on the Apache Hadoop YARN framework.
-
-## Configurations
-
-You can point the UI to custom locations by setting the environment variables in `src/main/webapp/config/configs.env`.
-
-In order to make the UI running on Ember server (started by `yarn start`)
-work with independently running ResouceManager,
-you need to enable CORS by setting `hadoop.http.cross-origin.enabled` to true
-and adding `org.apache.hadoop.security.HttpCrossOriginFilterInitializer`
-to `hadoop.http.filter.initializers` in core-site.xml of the ResourceManager.
-
-## Development
-
-All the following commands must be run inside `src/main/webapp`.
-
-### Prerequisites
-
-You will need the following things properly installed on your computer.
-
-* Install [Yarn](https://yarnpkg.com) v1.21.1
-* Install [Bower](http://bower.io/) v1.8.8
-* Install all dependencies by running `yarn install` & `bower install`
-
-### Running UI
-
-* `yarn start`
-* Visit your app at [http://localhost:4200](http://localhost:4200).
-
-### Building
-
-* `yarn run build` (production)
-* Files would be stored in "dist/"
-
-### Adding new dependencies
-
-**Warning: Do not edit the _package.json_ or _bower.json_ files manually. This could make them out-of-sync with the respective lock or shrinkwrap files.**
-
-YARN UI has replaced NPM with Yarn package manager. And hence Yarn would be used to manage dependencies defined in package.json.
-
-* Please use the Yarn and Bower command-line tools to add new dependencies. And the tool version must be same as those defined in Prerequisites section.
-* Once any dependency is added:
- * If it's in package.json. Make sure that the respective, and only those changes are reflected in yarn.lock file.
- * If it's in bower.json. Make sure that the respective, and only those changes are reflected in bower-shrinkwrap.json file.
-* Commands to add using CLI tools:
- * Yarn: yarn add [package-name]
- * Bower: bower install --save [package-name]
-
-### Adding new routes (pages), controllers, components etc.
-
-* Use ember-cli blueprint generator - [Ember CLI](http://ember-cli.com/extending/#generators-and-blueprints)
-
-### Building with Maven
-
-[YARN-6278](https://issues.apache.org/jira/browse/YARN-6278)
-added `yarn-ui` profile to pom.xml leveraging
-[frontend-maven-plugin](https://github.com/eirslett/frontend-maven-plugin) which
-automatically installs Node.js and Yarn locally under target/webapp directory.
-After building yarn-ui by `mvn package -Pyarn-ui`, you can reuse
-the locally installed Node.js and Yarn instead of manually installing them.
-
-```
-$ mvn package -Pyarn-ui
-$ export PATH=$PWD/target/webapp/node:$PATH
-$ export YARNJS=$PWD/target/webapp/node/yarn/dist/bin/yarn.js
-$ cd src/main/webapp/
-$ node $YARNJS install
-$ node node_modules/.bin/bower install
-$ node $YARNJS start
-```
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
deleted file mode 100644
index 58b4b3d534faa..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml
+++ /dev/null
@@ -1,304 +0,0 @@
-
-
-
-
- hadoop-yarn
- org.apache.hadoop
- 3.4.0-SNAPSHOT
-
- 4.0.0
- hadoop-yarn-ui
- 3.4.0-SNAPSHOT
- Apache Hadoop YARN UI
- ${packagingType}
-
-
- pom
- ${basedir}/target/webapp
- ${basedir}/target/webapp/node/node
- node/yarn/dist/bin/yarn.js
- false
-
-
-
-
-
-
- org.apache.rat
- apache-rat-plugin
-
-
- src/main/webapp/jsconfig.json
- src/main/webapp/bower.json
- src/main/webapp/bower-shrinkwrap.json
- src/main/webapp/package.json
- src/main/webapp/yarn.lock
- src/main/webapp/testem.json
- src/main/webapp/public/assets/images/**/*
- src/main/webapp/public/assets/images/*
- src/main/webapp/public/robots.txt
- public/crossdomain.xml
- src/main/webapp/.bowerrc
- src/main/webapp/.ember-cli
- src/main/webapp/.jshintrc
- src/main/webapp/.watchmanconfig
- src/main/webapp/WEB-INF/wro.xml
-
-
-
-
-
- maven-clean-plugin
-
- ${keepUIBuildCache}
- false
-
-
- ${webappDir}/bower_components
-
-
- ${webappDir}/node_modules
-
-
-
-
-
-
-
-
-
- yarn-ui
-
-
- false
-
-
-
- war
-
-
-
-
-
-
- maven-resources-plugin
-
-
- copy-resources
-
- validate
-
- copy-resources
-
-
- ${webappDir}
-
-
- ${basedir}/src/main/webapp
- true
-
- node_modules/**/*
- bower_components/**/*
- tmp/**/*
- dist/**/*
- terminal/**/*
-
-
-
-
-
-
- copy-terminal
-
- compile
-
- copy-resources
-
-
- ${webappDir}/dist
-
-
- ${basedir}/src/main/webapp
- true
-
- terminal/**/*
-
-
-
-
-
-
- copy-xterm
-
- compile
-
- copy-resources
-
-
- ${webappDir}/dist/terminal/xterm
-
-
- ${basedir}/target/webapp/node_modules/xterm
- true
-
- dist/**/*
-
-
-
-
-
-
-
-
-
-
- com.github.eirslett
- frontend-maven-plugin
- ${frontend-maven-plugin.version}
-
- ${webappDir}
-
-
-
-
-
- generate-resources
- install node and yarn
-
- install-node-and-yarn
-
-
- v8.17.0
- v1.21.1
-
-
-
- generate-resources
- yarn install
-
- yarn
-
-
- install
-
-
-
- generate-resources
- bower install
-
- install
-
-
- bower
-
-
-
-
-
-
- org.codehaus.mojo
- exec-maven-plugin
-
-
-
-
- ember build
- generate-resources
-
- exec
-
-
- ${webappDir}
- ${nodeExecutable}
-
- ${packageManagerScript}
- run
- build:mvn
-
-
-
-
-
-
-
-
-
- ro.isdc.wro4j
- wro4j-maven-plugin
- 1.7.9
-
-
-
- org.mockito
- mockito-core
- 2.18.0
-
-
-
-
- prepare-package
-
- run
-
-
-
-
- true
- yarn-ui,vendor
- ${basedir}/target/minified-resources/assets
- ${webappDir}/dist/assets
- ${webappDir}/WEB-INF/wro.xml
-
-
-
-
-
- org.apache.maven.plugins
- maven-war-plugin
-
-
- package
-
-
-
- ${webappDir}/WEB-INF/web.xml
- ${webappDir}/dist
-
-
- false
- ${basedir}/target/minified-resources
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-jar-plugin
-
- true
-
-
-
-
-
-
-
-
-
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.bowerrc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.bowerrc
deleted file mode 100644
index daf4462c44aca..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.bowerrc
+++ /dev/null
@@ -1,5 +0,0 @@
-{
- "directory": "bower_components",
- "analytics": false,
- "registry": "https://registry.bower.io"
-}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.ember-cli b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.ember-cli
deleted file mode 100644
index ee64cfed2a890..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.ember-cli
+++ /dev/null
@@ -1,9 +0,0 @@
-{
- /**
- Ember CLI sends analytics information by default. The data is completely
- anonymous, but there are times when you might want to disable this behavior.
-
- Setting `disableAnalytics` to true will prevent any data from being sent.
- */
- "disableAnalytics": false
-}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.gitignore b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.gitignore
deleted file mode 100644
index 082fb271f64a4..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.gitignore
+++ /dev/null
@@ -1,5 +0,0 @@
-tmp/
-node_modules/
-bower_components/
-dist/
-node/
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.jshintrc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.jshintrc
deleted file mode 100644
index 63cfe40a36853..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.jshintrc
+++ /dev/null
@@ -1,35 +0,0 @@
-{
- "predef": [
- "document",
- "window",
- "-Promise",
- "d3",
- "$",
- "moment"
- ],
- "browser": true,
- "boss": true,
- "curly": true,
- "debug": false,
- "devel": true,
- "eqeqeq": true,
- "evil": true,
- "forin": false,
- "immed": false,
- "laxbreak": false,
- "newcap": true,
- "noarg": true,
- "noempty": false,
- "nonew": false,
- "nomen": false,
- "onevar": false,
- "plusplus": false,
- "regexp": false,
- "undef": true,
- "sub": true,
- "strict": false,
- "white": false,
- "eqnull": true,
- "esnext": true,
- "unused": true
-}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.watchmanconfig b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.watchmanconfig
deleted file mode 100644
index e7834e3e4f39c..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/.watchmanconfig
+++ /dev/null
@@ -1,3 +0,0 @@
-{
- "ignore_dirs": ["tmp", "dist"]
-}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/WEB-INF/web.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/WEB-INF/web.xml
deleted file mode 100644
index ac74d5ccea770..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/WEB-INF/web.xml
+++ /dev/null
@@ -1,21 +0,0 @@
-
-
-
- YARN UI
-
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/WEB-INF/wro.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/WEB-INF/wro.xml
deleted file mode 100644
index 64c925adf88a0..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/WEB-INF/wro.xml
+++ /dev/null
@@ -1,9 +0,0 @@
-
-
-
- /vendor.js
-
-
- /yarn-ui.js
-
-
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/abstract.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/abstract.js
deleted file mode 100644
index 0db27a6e9a907..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/abstract.js
+++ /dev/null
@@ -1,49 +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.
- */
-import Ember from 'ember';
-import DS from 'ember-data';
-
-export default DS.JSONAPIAdapter.extend({
- address: null, //Must be set by inheriting classes
- restNameSpace: null, //Must be set by inheriting classes
- serverName: null, //Must be set by inheriting classes
-
- headers: {
- Accept: 'application/json'
- },
-
- host: Ember.computed("address", function () {
- var address = this.get("address");
- return this.get(`hosts.${address}`);
- }),
-
- namespace: Ember.computed("restNameSpace", function () {
- var serverName = this.get("restNameSpace");
- return this.get(`env.app.namespaces.${serverName}`);
- }),
-
- ajax: function(url, method, options) {
- options = options || {};
- options.crossDomain = true;
- options.xhrFields = {
- withCredentials: true
- };
- options.targetServer = this.get('serverName');
- return this._super(url, method, options);
- }
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/cluster-info.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/cluster-info.js
deleted file mode 100644
index 0665499685d08..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/cluster-info.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "rmWebAddress",
- restNameSpace: "cluster",
- serverName: "RM",
-
- // Any cluster-info specific adapter changes must be added here
- pathForType(/*modelName*/) {
- return ''; // move to some common place, return path by modelname.
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/cluster-metric.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/cluster-metric.js
deleted file mode 100644
index f431340d5deff..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/cluster-metric.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "rmWebAddress",
- restNameSpace: "metrics",
- serverName: "RM",
-
- // Any cluster-metric specific adapter changes must be added here
- pathForType(/*modelName*/) {
- return ''; // move to some common place, return path by modelname.
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/cluster-user-info.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/cluster-user-info.js
deleted file mode 100644
index a49c0f5edef83..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/cluster-user-info.js
+++ /dev/null
@@ -1,29 +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.
- */
-
-import RESTAbstractAdapter from './restabstract';
-
-export default RESTAbstractAdapter.extend({
- address: "rmWebAddress",
- restNameSpace: "cluster",
- serverName: "RM",
-
- pathForType(/*modelName*/) {
- return 'userinfo';
- }
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/jhs-health.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/jhs-health.js
deleted file mode 100644
index b55ac0b271d36..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/jhs-health.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-import RESTAbstractAdapter from './restabstract';
-
-export default RESTAbstractAdapter.extend({
- address: "jhsAddress",
- restNameSpace: "jhs",
- serverName: "JHS",
-
- urlForQueryRecord(/*query, modelName*/) {
- var url = this.buildURL();
- return url + '/info';
- }
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/restabstract.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/restabstract.js
deleted file mode 100644
index df409d63b0916..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/restabstract.js
+++ /dev/null
@@ -1,50 +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.
- */
-
-import DS from 'ember-data';
-import Ember from 'ember';
-
-export default DS.RESTAdapter.extend({
- address: null, //Must be set by inheriting classes
- restNameSpace: null, //Must be set by inheriting classes
- serverName: null, //Must be set by inheriting classes
-
- headers: {
- Accept: 'application/json'
- },
-
- host: Ember.computed("address", function() {
- var address = this.get("address");
- return this.get(`hosts.${address}`);
- }),
-
- namespace: Ember.computed("restNameSpace", function() {
- var nameSpace = this.get("restNameSpace");
- return this.get(`env.app.namespaces.${nameSpace}`);
- }),
-
- ajax(url, method, options) {
- options = options || {};
- options.crossDomain = true;
- options.xhrFields = {
- withCredentials: true
- };
- options.targetServer = this.get('serverName');
- return this._super(url, method, options);
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/timeline-health.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/timeline-health.js
deleted file mode 100644
index 8ca2310630642..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/timeline-health.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-import RESTAbstractAdapter from './restabstract';
-
-export default RESTAbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQueryRecord(/*query, modelName*/) {
- var url = this.buildURL();
- return url + '/health';
- }
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-attempt.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-attempt.js
deleted file mode 100644
index d5f3d0c2a4ea1..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-attempt.js
+++ /dev/null
@@ -1,40 +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.
- */
-
-import AbstractAdapter from './abstract';
-import Converter from 'yarn-ui/utils/converter';
-
-export default AbstractAdapter.extend({
- address: "rmWebAddress",
- restNameSpace: "cluster",
- serverName: "RM",
-
- urlForQuery(query/*, modelName*/) {
- var url = this._buildURL();
- var appId = query.appId;
- delete query.appId;
- return url + '/apps/' + appId + "/appattempts";
- },
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var url = this._buildURL();
- return url + '/apps/' +
- Converter.attemptIdToAppId(id) + "/appattempts/" + id;
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-flowrun.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-flowrun.js
deleted file mode 100644
index 23ecd1bae6d7d..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-flowrun.js
+++ /dev/null
@@ -1,33 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQuery(query/*, modelName*/){
- var url = this._buildURL();
- var flowrunUid = query['flowrunUid'];
- delete query.flowrunUid;
- url = url + '/run-uid/' + flowrunUid + '/apps?fields=ALL';
- return url;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-log.js
deleted file mode 100644
index 911e04062bcc2..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-log.js
+++ /dev/null
@@ -1,75 +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.
- */
-
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-import RESTAbstractAdapter from './restabstract';
-
-/**
- * REST URL's response when fetching container logs will be
- * in plain text format and not JSON.
- */
-export default RESTAbstractAdapter.extend({
- address: "jhsAddress",
- restNameSpace: "jhs",
- serverName: "JHS",
-
- headers: {
- Accept: 'text/plain'
- },
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var splits = Converter.splitForAppLogs(id);
- var containerId = splits[0];
- var logFile = splits[1];
- var url = this._buildURL();
- url = url + '/containerlogs/' + containerId + '/' + logFile + '?manual_redirection=true';
- Ember.Logger.info('The URL for getting the log: ' + url);
- return url;
- },
-
- handleResponse(status, headers, payload, requestData) {
- if (headers['location'] !== undefined && headers['location'] !== null) {
- return { redirectedUrl: headers.location, data: "" }
- } else {
- return { data: payload }
- }
- },
-
- /**
- * Override options so that result is not expected to be JSON
- */
- ajaxOptions: function (url, type, options) {
- var hash = options || {};
- hash.url = url;
- hash.type = type;
- // Make sure jQuery does not try to convert response to JSON.
- hash.dataType = 'text';
- hash.context = this;
-
- var headers = Ember.get(this, 'headers');
- if (headers !== undefined) {
- hash.beforeSend = function (xhr) {
- Object.keys(headers).forEach(function (key) {
- return xhr.setRequestHeader(key, headers[key]);
- });
- };
- }
- return hash;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-redirect-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-redirect-log.js
deleted file mode 100644
index 0f7e47d649632..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-jhs-redirect-log.js
+++ /dev/null
@@ -1,59 +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.
- */
-
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-import RESTAbstractAdapter from './restabstract';
-
-/**
- * REST URL's response when fetching container logs will be
- * in plain text format and not JSON.
- */
-export default RESTAbstractAdapter.extend({
- headers: {
- Accept: 'text/plain'
- },
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var splits = Converter.splitForAppLogs(id);
- var url = splits[0];
- return url;
- },
-
- /**
- * Override options so that result is not expected to be JSON
- */
- ajaxOptions: function (url, type, options) {
- var hash = options || {};
- hash.url = url;
- hash.type = type;
- // Make sure jQuery does not try to convert response to JSON.
- hash.dataType = 'text';
- hash.context = this;
-
- var headers = Ember.get(this, 'headers');
- if (headers !== undefined) {
- hash.beforeSend = function (xhr) {
- Object.keys(headers).forEach(function (key) {
- return xhr.setRequestHeader(key, headers[key]);
- });
- };
- }
- return hash;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-log.js
deleted file mode 100644
index e4ee6d11737a5..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-log.js
+++ /dev/null
@@ -1,79 +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.
- */
-
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-import RESTAbstractAdapter from './restabstract';
-
-/**
- * REST URL's response when fetching container logs will be
- * in plain text format and not JSON.
- */
-export default RESTAbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2Log",
- serverName: "ATS",
-
- headers: {
- Accept: 'text/plain'
- },
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var splits = Converter.splitForAppLogs(id);
- var clusterId = this.get("env.app.clusterId");
- var containerId = splits[0];
- var logFile = splits[1];
- if (splits[2]) {
- clusterId = splits[2];
- }
- var url = this._buildURL();
- url = url + '/containers/' + containerId + '/logs/' + logFile + '?clusterid=' + clusterId + '?manual_redirection=true';
- Ember.Logger.info('The URL for getting the log: ' + url);
- return url;
- },
-
- handleResponse(status, headers, payload, requestData) {
- if (headers['location'] !== undefined && headers['location'] !== null) {
- return { redirectedUrl: headers.location, data: "" }
- } else {
- return { data: payload }
- }
- },
-
- /**
- * Override options so that result is not expected to be JSON
- */
- ajaxOptions: function (url, type, options) {
- var hash = options || {};
- hash.url = url;
- hash.type = type;
- // Make sure jQuery does not try to convert response to JSON.
- hash.dataType = 'text';
- hash.context = this;
-
- var headers = Ember.get(this, 'headers');
- if (headers !== undefined) {
- hash.beforeSend = function (xhr) {
- Object.keys(headers).forEach(function (key) {
- return xhr.setRequestHeader(key, headers[key]);
- });
- };
- }
- return hash;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-redirect-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-redirect-log.js
deleted file mode 100644
index ba6accb2fdd18..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-redirect-log.js
+++ /dev/null
@@ -1,60 +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.
- */
-
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-import RESTAbstractAdapter from './restabstract';
-
-/**
- * REST URL's response when fetching container logs will be
- * in plain text format and not JSON.
- */
-export default RESTAbstractAdapter.extend({
-
- headers: {
- Accept: 'text/plain'
- },
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var splits = Converter.splitForAppLogs(id);
- var url = splits[0];
- return url;
- },
-
- /**
- * Override options so that result is not expected to be JSON
- */
- ajaxOptions: function (url, type, options) {
- var hash = options || {};
- hash.url = url;
- hash.type = type;
- // Make sure jQuery does not try to convert response to JSON.
- hash.dataType = 'text';
- hash.context = this;
-
- var headers = Ember.get(this, 'headers');
- if (headers !== undefined) {
- hash.beforeSend = function (xhr) {
- Object.keys(headers).forEach(function (key) {
- return xhr.setRequestHeader(key, headers[key]);
- });
- };
- }
- return hash;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-timeline.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-timeline.js
deleted file mode 100644
index f2bc30db20984..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app-timeline.js
+++ /dev/null
@@ -1,39 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var url = this._buildURL();
- var clusterId = this.get("env.app.clusterId");
- if (clusterId) {
- url += `/clusters/${clusterId}`;
- }
- url += '/apps/' + id + '?fields=ALL';
- return url;
- },
-
- pathForType(/*modelName*/) {
- return 'apps';
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app.js
deleted file mode 100644
index d29dff460d55a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-app.js
+++ /dev/null
@@ -1,50 +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.
- */
-
-import RESTAbstractAdapter from './restabstract';
-
-export default RESTAbstractAdapter.extend({
- address: "rmWebAddress",
- restNameSpace: "cluster",
- serverName: "RM",
-
- urlForQuery(/*query, modelName*/) {
- var url = this._buildURL();
- url = url + '/apps';
- return url;
- },
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var url = this._buildURL();
- url = url + '/apps/' + id;
- return url;
- },
-
- pathForType(/*modelName*/) {
- return 'apps'; // move to some common place, return path by modelname.
- },
-
- sendKillApplication(id) {
- var url = this._buildURL();
- url += '/apps/' + id + '/state';
- var data = {
- "state": "KILLED"
- };
- return this.ajax(url, "PUT", { data: data });
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-component-instance.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-component-instance.js
deleted file mode 100644
index 86e08fe37860b..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-component-instance.js
+++ /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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQuery(query/*, modelName*/) {
- var url = this.buildURL();
- var clusterId = this.get("env.app.clusterId");
- if (clusterId) {
- url += `/clusters/${clusterId}`;
- }
- url += '/apps/' + query.appId + '/entities/COMPONENT_INSTANCE?fields=ALL';
- delete query.appId;
- return url;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-conf.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-conf.js
deleted file mode 100644
index 662197216f452..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-conf.js
+++ /dev/null
@@ -1,79 +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.
- */
-
- import DS from 'ember-data';
- import Ember from 'ember';
- import Converter from 'yarn-ui/utils/converter';
- import AbstractAdapter from './abstract';
-
- export default AbstractAdapter.extend({
-
- address: 'rmWebAddress',
-
- headers: {
- Accept: 'text/plain'
- },
-
- host: Ember.computed("address", function () {
- let address = this.get("address");
- return this.get(`hosts.${address}`);
- }),
-
- pathForType(type) {
- return 'conf';
- },
-
- urlForFindRecord(id, modelName, snapshot) {
- var extension = this.get("host").split('/').pop();
- if (extension != id) {
- this.host = this.get("host") + id;
- }
- var url = this._buildURL();
- return url;
- },
-
- ajax(url, method, hash) {
- hash = hash || {};
- hash.crossDomain = true;
- hash.xhrFields = {withCredentials: true};
- hash.targetServer = "RM";
- return this._super(url, method, hash);
- },
-
- /**
- * Override options so that result is not expected to be JSON
- */
- ajaxOptions: function (url, type, options) {
- var hash = options || {};
- hash.url = url;
- hash.type = type;
- // Make sure jQuery does not try to convert response to JSON.
- hash.dataType = 'text';
- hash.context = this;
-
- var headers = Ember.get(this, 'headers');
- if (headers != undefined) {
- hash.beforeSend = function (xhr) {
- Object.keys(headers).forEach(function (key) {
- return xhr.setRequestHeader(key, headers[key]);
- });
- };
- }
- return hash;
- },
- });
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-container-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-container-log.js
deleted file mode 100644
index df461278c1e60..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-container-log.js
+++ /dev/null
@@ -1,80 +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.
- */
-
-import DS from 'ember-data';
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-
-/**
- * REST URL's response when fetching container logs will be
- * in plain text format and not JSON.
- */
-export default DS.RESTAdapter.extend({
- headers: {
- Accept: 'text/plain'
- },
-
- host: Ember.computed("address", function () {
- return this.get(`hosts.localBaseAddress`);
- }),
-
- namespace: Ember.computed("restNameSpace", function () {
- return this.get(`env.app.namespaces.node`);
- }),
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var splits = Converter.splitForContainerLogs(id);
- var nodeHttpAddr = splits[0];
- var containerId = splits[1];
- var filename = splits[2];
- var url = this._buildURL();
- url = url.replace("{nodeAddress}", nodeHttpAddr) + "/containerlogs/"
- + containerId + "/" + filename;
- return url;
- },
-
- ajax(url, method, hash) {
- hash = hash || {};
- hash.crossDomain = true;
- hash.xhrFields = {withCredentials: true};
- hash.targetServer = "NM";
- return this._super(url, method, hash);
- },
-
- /**
- * Override options so that result is not expected to be JSON
- */
- ajaxOptions: function (url, type, options) {
- var hash = options || {};
- hash.url = url;
- hash.type = type;
- // Make sure jQuery does not try to convert response to JSON.
- hash.dataType = 'text';
- hash.context = this;
-
- var headers = Ember.get(this, 'headers');
- if (headers !== undefined) {
- hash.beforeSend = function (xhr) {
- Object.keys(headers).forEach(function (key) {
- return xhr.setRequestHeader(key, headers[key]);
- });
- };
- }
- return hash;
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-container.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-container.js
deleted file mode 100644
index 476c9bec19671..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-container.js
+++ /dev/null
@@ -1,49 +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.
- */
-
-import DS from 'ember-data';
-import Converter from 'yarn-ui/utils/converter';
-
-export default DS.JSONAPIAdapter.extend({
- headers: {
- Accept: 'application/json'
- },
-
- host: function() {
- return this.get(`hosts.rmWebAddress`);
- }.property(),
-
- namespace: function() {
- return this.get(`env.app.namespaces.cluster`);
- }.property(),
-
- urlForQuery(query/*, modelName*/) {
- var url = this._buildURL();
- url = url + '/apps/' + Converter.attemptIdToAppId(query.app_attempt_id) +
- "/appattempts/" + query.app_attempt_id + "/containers";
- return url;
- },
-
- ajax(url, method, hash) {
- hash = {};
- hash.crossDomain = true;
- hash.xhrFields = {withCredentials: true};
- hash.targetServer = "RM";
- return this._super(url, method, hash);
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-entity.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-entity.js
deleted file mode 100644
index 3d7338f890509..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-entity.js
+++ /dev/null
@@ -1,35 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQuery(query/*, modelName*/){
- var url = this._buildURL();
- var appUid = query.app_uid;
- var entityType = query.entity_type;
- delete query.app_uid;
- delete query.entity_type;
- url = url + '/app-uid/' + appUid + '/entities/' + entityType + '?fields=INFO';
- return url;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-flow-activity.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-flow-activity.js
deleted file mode 100644
index cb8dfee20fc18..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-flow-activity.js
+++ /dev/null
@@ -1,34 +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.
- */
-
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- pathForType(/*modelName*/) {
- var clusterId = this.get("env.app.clusterId");
- if (clusterId) {
- return `clusters/${clusterId}/flows`;
- }
- return 'flows'; // move to some common place, return path by modelname.
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-flowrun-brief.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-flowrun-brief.js
deleted file mode 100644
index ea9c6cb49696b..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-flowrun-brief.js
+++ /dev/null
@@ -1,32 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQuery(query/*, modelName*/){
- var url = this._buildURL();
- var flowuid = query['flowuid'];
- delete query.flowuid;
- return url + '/flow-uid/' + flowuid + '/runs?fields=ALL';
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-flowrun.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-flowrun.js
deleted file mode 100644
index 9163c83625fdd..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-flowrun.js
+++ /dev/null
@@ -1,38 +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.
- */
-
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQueryRecord(query/*, modelName*/){
- var url = this._buildURL();
- var flowrunuid = query.flowrun_uid;
- delete query.flowrun_uid;
- url = url + '/run-uid/' + flowrunuid;
- return url;
- },
-
- pathForType(/*modelName*/) {
- return 'run-uid'; // move to some common place, return path by modelname.
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-container.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-container.js
deleted file mode 100644
index aeab48de52db6..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-container.js
+++ /dev/null
@@ -1,33 +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.
- */
-
-import Ember from 'ember';
-import RESTAbstractAdapter from './restabstract';
-
-export default RESTAbstractAdapter.extend({
- address: "jhsAddress",
- restNameSpace: "jhs",
- serverName: "JHS",
-
- urlForQuery(query/*, modelName*/) {
- var url = this.buildURL();
- url = url + '/aggregatedlogs?appattemptid=' + query.app_attempt_id;
- delete query.app_attempt_id;
- return url;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-log.js
deleted file mode 100644
index 97fe57bb9585a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-log.js
+++ /dev/null
@@ -1,42 +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.
- */
-
-import AbstractAdapter from './abstract';
-import { createEmptyContainerLogInfo } from 'yarn-ui/helpers/log-adapter-helper';
-
-export default AbstractAdapter.extend({
- address: "jhsAddress",
- restNameSpace: "jhs",
- serverName: "JHS",
-
- urlForQuery(query/*, modelName*/) {
- var url = this._buildURL();
- var containerId = query['containerId'];
- delete query.containerId;
- return url + '/containers/' + containerId + '/logs' + '?manual_redirection=true';
- },
-
- handleResponse(status, headers, payload, requestData) {
- if (headers['location'] !== undefined && headers['location'] !== null) {
- return createEmptyContainerLogInfo(headers['location']);
- } else {
- return payload;
- }
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-redirect-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-redirect-log.js
deleted file mode 100644
index b3bb66e8a471e..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-jhs-redirect-log.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "jhsAddress",
- restNameSpace: "jhs",
- serverName: "JHS",
-
- urlForQuery(url/*, modelName*/) {
- return url;
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-log.js
deleted file mode 100644
index 171dda740dff6..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-log.js
+++ /dev/null
@@ -1,55 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-function createEmptyContainerLogInfo(location) {
- return {
- containerLogsInfo: {
- containerLogInfo: [{
- fileName: "",
- fileSize: "",
- lastModifiedTime: "",
- redirectedUrl: location
- }]
- }
- };
-}
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2Log",
- serverName: "ATS",
-
- urlForQuery(query/*, modelName*/) {
- var url = this._buildURL();
- var containerId = query['containerId'];
- var clusterId = this.get("env.app.clusterId");
- delete query.containerId;
- return url + '/containers/' + containerId + '/logs' + '?clusterid=' + clusterId + '?manual_redirection=true';
- },
-
- handleResponse(status, headers, payload, requestData) {
- if (headers['location'] !== undefined && headers['location'] !== null) {
- return createEmptyContainerLogInfo(headers['location']);
- } else {
- return payload;
- }
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-metrics.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-metrics.js
deleted file mode 100644
index d26de2819e33f..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-metrics.js
+++ /dev/null
@@ -1,76 +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.
- */
-
-import DS from 'ember-data';
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-
-export default DS.RESTAdapter.extend({
- address: 'rmWebAddress',
-
- headers: {
- Accept: 'text/plain'
- },
-
- host: Ember.computed("address", function() {
- let address = this.get("address");
- return this.get(`hosts.${address}`);
- }),
-
- pathForType(type) {
- return 'jmx';
- },
-
- urlForFindRecord(id, modelName, snapshot) {
- var extension = this.get("host").split('/').pop();
- if (extension != id) {
- this.host = this.get("host") + id;
- }
- var url = this._buildURL();
- return url;
- },
-
- ajax(url, method, hash) {
- hash = hash || {};
- hash.crossDomain = true;
- hash.xhrFields = {withCredentials: true};
- hash.targetServer = "RM";
- return this._super(url, method, hash);
- },
-
- /**
- * Override options so that result is not expected to be JSON
- */
- ajaxOptions: function (url, type, options) {
- var hash = options || {};
- hash.url = url;
- hash.type = type;
- // Make sure jQuery does not try to convert response to JSON.
- hash.dataType = 'text';
- hash.context = this;
- var headers = Ember.get(this, 'headers');
- if (headers != undefined) {
- hash.beforeSend = function (xhr) {
- Object.keys(headers).forEach(function (key) {
- return xhr.setRequestHeader(key, headers[key]);
- });
- };
- }
- return hash;
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-nm-gpu.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-nm-gpu.js
deleted file mode 100644
index bf6307a664c0e..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-nm-gpu.js
+++ /dev/null
@@ -1,33 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
-
- address: "localBaseAddress",
- restNameSpace: "node",
- serverName: "NM",
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var url = this._buildURL();
- url = url.replace("{nodeAddress}", id) + "/resources/yarn.io%2Fgpu";
- return url;
- }
-
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-node-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-node-app.js
deleted file mode 100644
index 6d69828069e58..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-node-app.js
+++ /dev/null
@@ -1,53 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
-
- address: "localBaseAddress",
- restNameSpace: "node",
- serverName: "NM",
-
- urlForQuery(query) {
- var url = this._buildURL();
- url = url.replace("{nodeAddress}", query.nodeAddr) + "/apps";
- return url;
- },
-
- urlForQueryRecord: function (query) {
- var url = this._buildURL();
- url = url.replace("{nodeAddress}", query.nodeAddr) + "/apps/" + query.appId;
- return url;
- },
-
- query: function (store, type, query) {
- var url = this.urlForQuery(query);
- // Query params not required.
- query = null;
- return this.ajax(url, 'GET', { data: query });
- },
-
- queryRecord: function (store, type, query) {
- var url = this.urlForQueryRecord(query);
- // Query params not required.
- query = null;
- return this.ajax(url, 'GET', { data: query });
- },
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-node-container.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-node-container.js
deleted file mode 100644
index 0fff047165dde..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-node-container.js
+++ /dev/null
@@ -1,54 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
-
- address: "localBaseAddress",
- restNameSpace: "node",
- serverName: "NM",
-
- urlForQuery(query) {
- var url = this._buildURL();
- url = url.replace("{nodeAddress}", query.nodeHttpAddr) + "/containers";
- return url;
- },
-
- urlForQueryRecord(query) {
- var url = this._buildURL();
- url = url.replace("{nodeAddress}", query.nodeHttpAddr) + "/containers/" + query.containerId;
- return url;
- },
-
- query: function (store, type, query) {
- var url = this.urlForQuery(query);
- // Query params not required.
- query = null;
- return this.ajax(url, 'GET', { data: query });
- },
-
- queryRecord: function (store, type, query) {
- var url = this.urlForQueryRecord(query);
- // Query params not required.
- query = null;
- console.log(url);
- return this.ajax(url, 'GET', { data: query });
- },
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-node.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-node.js
deleted file mode 100644
index 1436bc13f2bc3..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-node.js
+++ /dev/null
@@ -1,33 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
-
- address: "localBaseAddress",
- restNameSpace: "node",
- serverName: "NM",
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var url = this._buildURL();
- url = url.replace("{nodeAddress}", id);
- return url;
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js
deleted file mode 100644
index 7eb9f76b0aff3..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js
+++ /dev/null
@@ -1,23 +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.
- */
-
-import YarnQueueAdapter from './yarn-queue';
-
-export default YarnQueueAdapter.extend({
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js
deleted file mode 100644
index 7eb9f76b0aff3..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js
+++ /dev/null
@@ -1,23 +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.
- */
-
-import YarnQueueAdapter from './yarn-queue';
-
-export default YarnQueueAdapter.extend({
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js
deleted file mode 100644
index 7eb9f76b0aff3..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js
+++ /dev/null
@@ -1,23 +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.
- */
-
-import YarnQueueAdapter from './yarn-queue';
-
-export default YarnQueueAdapter.extend({
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js
deleted file mode 100644
index 8184c39cf09db..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-import AbstractAdapter from '../abstract';
-
-export default AbstractAdapter.extend({
- address: "rmWebAddress",
- restNameSpace: "cluster",
- serverName: "RM",
-
- pathForType(/*modelName*/) {
- return 'scheduler'; // move to some common place, return path by modelname.
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-redirect-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-redirect-log.js
deleted file mode 100644
index 7c2b81e025b75..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-redirect-log.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2Log",
- serverName: "ATS",
-
- urlForQuery(url/*, modelName*/) {
- return url;
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-log.js
deleted file mode 100644
index a912ffeaab6f3..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-log.js
+++ /dev/null
@@ -1,76 +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.
- */
-
-import DS from 'ember-data';
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-
-export default DS.RESTAdapter.extend({
- address: 'rmWebAddress',
- headers: {
- Accept: 'text/plain'
- },
-
- host: Ember.computed("address", function() {
- let address = this.get("address");
- return this.get(`hosts.${address}`);
- }),
-
- pathForType(type) {
- return 'logs';
- },
-
- buildURL (modelName, id, snapshot, requestType, query) {
- return this._super(modelName, id, snapshot, requestType, query) + '/';
- },
-
- urlForFindRecord(id, modelName, snapshot) {
- this.host = this.get('host');
- let url = this.host + id;
- return url;
- },
-
- ajax(url, method, hash) {
- hash = hash || {};
- hash.crossDomain = true;
- hash.xhrFields = {withCredentials: true};
- hash.targetServer = "RM";
- return this._super(url, method, hash);
- },
-
- /**
- * Override options so that result is not expected to be JSON
- */
- ajaxOptions: function (url, type, options) {
- var hash = options || {};
- hash.url = url;
- hash.type = type;
- // Make sure jQuery does not try to convert response to JSON.
- hash.dataType = 'text';
- hash.context = this;
- var headers = Ember.get(this, 'headers');
- if (headers != undefined) {
- hash.beforeSend = function (xhr) {
- Object.keys(headers).forEach(function (key) {
- return xhr.setRequestHeader(key, headers[key]);
- });
- };
- }
- return hash;
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-node.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-node.js
deleted file mode 100644
index 6783ded87b158..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-rm-node.js
+++ /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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "rmWebAddress",
- restNameSpace: "cluster",
- serverName: "RM",
-
- pathForType(/*modelName*/) {
- return 'nodes';
- },
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var url = this._buildURL();
- url = url + "/nodes/" + id;
- return url;
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-service-component.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-service-component.js
deleted file mode 100644
index 4fda603e11aca..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-service-component.js
+++ /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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQuery(query/*, modelName*/) {
- var url = this.buildURL();
- var clusterId = this.get("env.app.clusterId");
- if (clusterId) {
- url += `/clusters/${clusterId}`;
- }
- url += '/apps/' + query.appId + '/entities/COMPONENT?fields=ALL';
- delete query.appId;
- return url;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-service-info.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-service-info.js
deleted file mode 100644
index 9e3775489a9eb..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-service-info.js
+++ /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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQueryRecord(query/*, modelName*/) {
- var url = this.buildURL();
- var clusterId = this.get("env.app.clusterId");
- if (clusterId) {
- url += `/clusters/${clusterId}`;
- }
- url += '/apps/' + query.appId + '/entities/SERVICE_ATTEMPT?fields=ALL';
- delete query.appId;
- return url;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-service.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-service.js
deleted file mode 100644
index 221281047496a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-service.js
+++ /dev/null
@@ -1,32 +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.
- */
-
-import RESTAbstractAdapter from './restabstract';
-
-export default RESTAbstractAdapter.extend({
- address: "rmWebAddress",
- restNameSpace: "dashService",
- serverName: "DASH",
-
- urlForQueryRecord(query/*, modelName*/) {
- var url = this.buildURL();
- url += '/' + query.serviceName;
- delete query.serviceName;
- return url;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-servicedef.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-servicedef.js
deleted file mode 100644
index 954aafc784231..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-servicedef.js
+++ /dev/null
@@ -1,58 +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.
- */
-
-import RESTAbstractAdapter from './restabstract';
-
-export default RESTAbstractAdapter.extend({
- address: "rmWebAddress",
- restNameSpace: "dashService",
- serverName: "DASH",
-
- normalizeErrorResponse(status, headers, payload) {
- if (payload && typeof payload === 'object' && payload.errors) {
- return payload.errors;
- } else {
- return [
- payload
- ];
- }
- },
-
- deployService(request, user) {
- var url = this.buildURL();
- if(user) {
- url += "/?user.name=" + user;
- }
- return this.ajax(url, "POST", {data: request});
- },
-
- stopService(serviceName, user) {
- var url = this.buildURL();
- url += "/" + serviceName;
- url += "/?user.name=" + user;
- var data = {"state": "STOPPED", "name": serviceName};
- return this.ajax(url, "PUT", {data: data});
- },
-
- deleteService(serviceName, user) {
- var url = this.buildURL();
- url += "/" + serviceName;
- url += "/?user.name=" + user;
- return this.ajax(url, "DELETE", {data: {}});
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-timeline-appattempt.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-timeline-appattempt.js
deleted file mode 100644
index cbc10bd68905a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-timeline-appattempt.js
+++ /dev/null
@@ -1,49 +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.
- */
-
-import AbstractAdapter from './abstract';
-import Converter from 'yarn-ui/utils/converter';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQuery(query/*, modelName*/) {
- var url = this._buildURL();
- var clusterId = this.get("env.app.clusterId")
- if (clusterId) {
- url += `/clusters/${clusterId}`;
- }
- var appId = query.appId;
- query.fields = 'ALL';
- delete query.appId;
- return url + '/apps/' + appId + "/entities/YARN_APPLICATION_ATTEMPT";
- },
-
- urlForFindRecord(id/*, modelName, snapshot*/) {
- var url = this._buildURL();
- var clusterId = this.get("env.app.clusterId")
- if (clusterId) {
- url += `/clusters/${clusterId}`;
- }
- return url + '/apps/' + Converter.attemptIdToAppId(id) +
- "/entities/YARN_APPLICATION_ATTEMPT/" + id + "?fields=ALL";
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-timeline-container.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-timeline-container.js
deleted file mode 100644
index 546b03f46277c..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-timeline-container.js
+++ /dev/null
@@ -1,44 +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.
- */
-
-import AbstractAdapter from './abstract';
-import Converter from 'yarn-ui/utils/converter';
-
-export default AbstractAdapter.extend({
- address: "timelineWebAddress",
- restNameSpace: "timelineV2",
- serverName: "ATS",
-
- urlForQuery(query/*, modelName*/){
- var url = this._buildURL();
- var clusterId = this.get("env.app.clusterId")
- if (clusterId) {
- url += `/clusters/${clusterId}`;
- }
- var app_attempt_id = query.app_attempt_id;
- query.fields = 'ALL';
- delete query.app_attempt_id;
- url = url + '/apps/' + Converter.attemptIdToAppId(app_attempt_id) +
- '/entities/YARN_CONTAINER';
- return url;
- },
-
- pathForType(/*modelName*/) {
- return '';
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/app.js
deleted file mode 100644
index 8c1025ae7269c..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/app.js
+++ /dev/null
@@ -1,38 +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.
- */
-
-import Ember from 'ember';
-import Resolver from 'ember-resolver';
-import loadInitializers from 'ember/load-initializers';
-import config from './config/environment';
-import Sorter from 'yarn-ui/utils/sorter';
-
-var App;
-
-Ember.MODEL_FACTORY_INJECTIONS = true;
-
-App = Ember.Application.extend({
- modulePrefix: config.modulePrefix,
- podModulePrefix: config.podModulePrefix,
- Resolver
-});
-
-loadInitializers(App, config.modulePrefix);
-Sorter.initDataTableSorter();
-
-export default App;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-attempt-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-attempt-table.js
deleted file mode 100644
index 88282750e2391..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-attempt-table.js
+++ /dev/null
@@ -1,22 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-table.js
deleted file mode 100644
index 4b741b886279a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-table.js
+++ /dev/null
@@ -1,22 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-timeout-bar.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-timeout-bar.js
deleted file mode 100644
index 0eac82759f212..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-timeout-bar.js
+++ /dev/null
@@ -1,60 +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.
- */
-
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-
-export default Ember.Component.extend({
- app: null,
-
- appTimeoutValue: function() {
- var timeoutValueInSecs = this.get("app.remainingTimeoutInSeconds");
- if (timeoutValueInSecs > -1) {
- return Converter.msToElapsedTime(timeoutValueInSecs * 1000);
- } else {
- return timeoutValueInSecs;
- }
- }.property("app.remainingTimeoutInSeconds"),
-
- isAppTimedOut: function() {
- if (this.get("app.remainingTimeoutInSeconds") > 0) {
- return false;
- } else {
- return true;
- }
- }.property("app.remainingTimeoutInSeconds"),
-
- appTimeoutBarStyle: function() {
- var remainingInSecs = this.get("app.remainingTimeoutInSeconds"),
- expiryTimestamp = Converter.dateToTimeStamp(this.get("app.applicationExpiryTime")),
- expiryInSecs = expiryTimestamp / 1000,
- startTimestamp = Converter.dateToTimeStamp(this.get("app.startTime")),
- startInSecs = startTimestamp / 1000,
- totalRunInSecs = 0,
- appRunDurationInSecs = 0,
- width = 0;
-
- if (remainingInSecs > 0) {
- totalRunInSecs = expiryInSecs - startInSecs;
- appRunDurationInSecs = totalRunInSecs - remainingInSecs;
- width = appRunDurationInSecs / totalRunInSecs * 100;
- }
-
- return "width: " + width + "%";
- }.property("app.remainingTimeoutInSeconds", "app.applicationExpiryTime", "app.startTime")
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-usage-donut-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-usage-donut-chart.js
deleted file mode 100644
index c72d93451ef3d..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/app-usage-donut-chart.js
+++ /dev/null
@@ -1,59 +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.
- */
-
-import BaseUsageDonutChart from 'yarn-ui/components/base-usage-donut-chart';
-import ColorUtils from 'yarn-ui/utils/color-utils';
-import HrefAddressUtils from 'yarn-ui/utils/href-address-utils';
-
-export default BaseUsageDonutChart.extend({
- colors: d3.scale.category20().range(),
-
- draw: function() {
- var usageByApps = [];
- var avail = 100;
-
- this.get("data").forEach(function (app) {
- var v = app.get("clusterUsagePercentage");
- if (v > 1e-2) {
- usageByApps.push({
- label: app.get("id"),
- link: HrefAddressUtils.getApplicationLink(app.get("id")),
- value: v.toFixed(2)
- });
-
- avail = avail - v;
- }
- }.bind(this));
-
- usageByApps.sort(function(a,b) {
- return b.value - a.value;
- });
-
- usageByApps = this.mergeLongTails(usageByApps, 8);
-
- usageByApps.push({
- label: "Available",
- value: avail.toFixed(4)
- });
-
- this.colors = ColorUtils.getColors(usageByApps.length, ["others", "good"], true);
-
- this.renderDonutChart(usageByApps, this.get("title"), this.get("showLabels"),
- this.get("middleLabel"), "100%", "%");
- },
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/bar-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/bar-chart.js
deleted file mode 100644
index 05d78eb902f14..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/bar-chart.js
+++ /dev/null
@@ -1,123 +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.
- */
-
-import Ember from 'ember';
-import BaseChartComponent from 'yarn-ui/components/base-chart-component';
-
-export default BaseChartComponent.extend({
- // data:
- // [{label=label1, value=value1}, ...]
- // ...
- renderBarChart: function(data, title, textWidth = 50) {
- var g = this.chart.g;
- var layout = this.getLayout();
- this.renderTitleAndBG(g, title, layout);
-
- var maxValue = -1;
- for (var i = 0; i < data.length; i++) {
- if (data[i] instanceof Array) {
- if (data[i][0].value > maxValue) {
- maxValue = data[i][0].value;
- }
- } else {
- if (data[i].value > maxValue) {
- maxValue = data[i].value;
- }
- }
- }
-
- var singleBarHeight = 30;
-
- // 50 is for text
- var maxBarWidth = layout.x2 - layout.x1 - 2 * layout.margin - textWidth - 50;
-
- // 30 is for title
- var maxBarsHeight = layout.y2 - layout.y1 - 2 * layout.margin - 30;
- var gap = (maxBarsHeight - data.length * singleBarHeight) / (data.length -
- 1);
-
- var xScaler = d3.scale.linear()
- .domain([0, maxValue])
- .range([0, maxBarWidth]);
-
- var getBarText = function(i) {
- return data[i].label;
- };
- // show bar text
- for (i = 0; i < data.length; i++) {
- g.append("text")
- .text(getBarText(i))
- .attr("y", layout.y1 + singleBarHeight / 2 + layout.margin +
- (gap + singleBarHeight) * i + 30)
- .attr("x", layout.x1 + layout.margin);
- }
-
- // show bar
- var bar = g.selectAll("bars")
- .data(data)
- .enter()
- .append("rect")
- .attr("y", function(d, i) {
- return layout.y1 + 30 + layout.margin + (gap + singleBarHeight) * i;
- })
- .attr("x", layout.x1 + layout.margin + textWidth)
- .attr("height", singleBarHeight)
- .attr("fill", function(d, i) {
- return this.colors[i];
- }.bind(this))
- .attr("width", 0);
-
- this.bindTooltip(bar);
-
- bar.transition()
- .duration(500)
- .attr("width", function(d) {
- var w;
- w = xScaler(d.value);
- // At least each item has 3 px
- w = Math.max(w, 3);
- return w;
- });
-
- var getBarValue = function(i) {
- return data[i].value;
- };
- // show bar value
- for (i = 0; i < data.length; i++) {
- g.append("text")
- .text(getBarValue(i))
- .attr("y", layout.y1 + singleBarHeight / 2 + layout.margin +
- (gap + singleBarHeight) * i + 30)
- .attr("x", layout.x1 + layout.margin + textWidth + 15 + xScaler(data[i].value));
- }
- },
-
- draw: function() {
- this.renderBarChart(this.get("data"), this.get("title"), this.get("textWidth"));
- },
-
- _dataChange: Ember.observer("data", function() {
- this.chart.g.selectAll("*").remove();
- this.draw();
- }),
-
- didInsertElement: function() {
- this.initChart();
- this.draw();
- }
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-chart-component.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-chart-component.js
deleted file mode 100644
index 26aa2b051051f..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-chart-component.js
+++ /dev/null
@@ -1,148 +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.
- */
-
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-
-export default Ember.Component.extend({
- tooltip : undefined,
- colors: d3.scale.category10().range(),
-
- init: function () {
- this._super();
- this.set("chart", {
- svg: undefined,
- g: undefined,
- h: 0,
- w: 0,
- tooltip: undefined
- });
- },
-
- initChart: function(removeLast = false) {
- // Init tooltip if it is not initialized
- // this.tooltip = d3.select("#chart-tooltip");
- if (!this.tooltip) {
- this.tooltip = d3.select("body")
- .append("div")
- .attr("class", "tooltip")
- .attr("id", "chart-tooltip")
- .style("opacity", 0);
- }
-
- var parentId = this.get("parentId");
-
- if (removeLast) {
- // Init svg
- var svg = d3.select("#" + parentId + "-svg");
- if (svg) {
- svg.remove();
- }
- }
-
- var parent = d3.select("#" + parentId);
- var bbox = parent.node().getBoundingClientRect();
- this.chart.w = bbox.width - 30;
-
- var ratio = 0.75; // 4:3 by default
- if (this.get("ratio")) {
- ratio = this.get("ratio");
- }
- this.chart.h = bbox.width * ratio;
-
- if (this.get("maxHeight")) {
- this.chart.h = Math.min(this.get("maxHeight"), this.chart.h);
- }
-
- this.chart.svg = parent.append("svg")
- .attr("width", this.chart.w)
- .attr("height", this.chart.h)
- .attr("id", parentId + "-svg");
-
- this.chart.g = this.chart.svg.append("g");
- },
-
- renderTitleAndBG: function(g, title, layout, background=true) {
- var bg = g.append("g");
- bg.append("text")
- .text(title)
- .attr("x", (layout.x1 + layout.x2) / 2)
- .attr("y", layout.y1 + layout.margin + 20)
- .attr("class", "chart-title");
-
- if (background) {
- bg.append("rect")
- .attr("x", layout.x1)
- .attr("y", layout.y1)
- .attr("width", layout.x2 - layout.x1)
- .attr("height", layout.y2 - layout.y1)
- .attr("class", "chart-frame");
- }
- },
-
- bindTooltip: function(d) {
- d.on("mouseover", function() {
- this.tooltip
- .style("left", (d3.event.pageX) + "px")
- .style("top", (d3.event.pageY - 28) + "px");
- }.bind(this))
- .on("mousemove", function(d) {
- // Handle pie chart case
- var data = d;
- if (d.data) {
- data = d.data;
- }
-
- this.tooltip.style("opacity", 0.7);
- var value = data.value;
- if (this.get("type") === "memory") {
- value = Converter.memoryToSimpliedUnit(value);
- }
- this.tooltip.html(data.label + " = " + value)
- .style("left", (d3.event.pageX) + "px")
- .style("top", (d3.event.pageY - 28) + "px");
- }.bind(this))
- .on("mouseout", function() {
- this.tooltip.style("opacity", 0);
- }.bind(this));
- },
-
- adjustMaxHeight: function(h) {
- this.chart.svg.attr("height", h);
- },
-
- getLayout: function() {
- var x1 = 0;
- var y1 = 0;
- var x2 = this.chart.w;
- var y2 = this.chart.h;
-
- var layout = {
- x1: x1,
- y1: y1,
- x2: x2 - 10,
- y2: y2 - 10,
- margin: 10
- };
- return layout;
- },
-
- willDestroy: function() {
- this.tooltip.remove();
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-usage-donut-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-usage-donut-chart.js
deleted file mode 100644
index 201ae6ca85e56..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/base-usage-donut-chart.js
+++ /dev/null
@@ -1,42 +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.
- */
-
-import DonutChart from 'yarn-ui/components/donut-chart';
-
-export default DonutChart.extend({
- mergeLongTails: function(usages, nItemsKept) {
- var arr = [];
- for (var i = 0; i < Math.min(usages.length, nItemsKept); i++) {
- arr.push(usages[i]);
- }
-
- var others = {
- label: "Used by others",
- value: 0
- };
-
- for (i = nItemsKept; i < usages.length; i++) {
- others.value += Number(usages[i].value);
- }
- others.value = others.value.toFixed(2);
-
- arr.push(others);
-
- return arr;
- }
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/breadcrumb-bar.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/breadcrumb-bar.js
deleted file mode 100644
index b8d974aa85025..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/breadcrumb-bar.js
+++ /dev/null
@@ -1,32 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
-
- breadcrumbs: null,
- hideRefresh: false,
-
- actions:{
- refresh: function () {
- this.get("targetObject").send("refresh");
- }
- }
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/collapsible-panel.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/collapsible-panel.js
deleted file mode 100644
index 6a40f8f271431..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/collapsible-panel.js
+++ /dev/null
@@ -1,64 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- classNames: ['pull-right'],
-
- targetId: '',
- initialClosedState: false,
-
- didInsertElement() {
- if (!this.get('targetId')) {
- this.$('.toggle_switch').hide();
- }
- if (this.get('targetId') && this.get('initialClosedState')) {
- this.$('.toggle_switch').show();
- this.toggleToggleSwitchArrow();
- Ember.$('#' + this.get('targetId')).removeClass('panel-collapsed').show();
- }
- },
-
- toggleToggleSwitchArrow() {
- let $toggleArrow = this.$('.toggle_switch').find('span');
- if ($toggleArrow.hasClass('glyphicon-chevron-up')) {
- $toggleArrow.removeClass('glyphicon-chevron-up').addClass('glyphicon-chevron-down');
- } else {
- $toggleArrow.removeClass('glyphicon-chevron-down').addClass('glyphicon-chevron-up');
- }
- },
-
- toggleCollapsiblePanel() {
- let $collapsiblePanel = Ember.$('#' + this.get('targetId'));
- if ($collapsiblePanel.hasClass('panel-collapsed')) {
- $collapsiblePanel.removeClass('panel-collapsed');
- $collapsiblePanel.slideDown();
- } else {
- $collapsiblePanel.addClass('panel-collapsed');
- $collapsiblePanel.slideUp();
- }
- },
-
- actions: {
- togglePanelCollapse() {
- this.toggleToggleSwitchArrow();
- this.toggleCollapsiblePanel();
- }
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/confirm-dialog.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/confirm-dialog.js
deleted file mode 100644
index a6f518b921f37..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/confirm-dialog.js
+++ /dev/null
@@ -1,31 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- dialogId: "confirmModalDialog",
- title: "Confirm",
- message: "Are you sure?",
-
- actions: {
- yesConfirmed() {
- this.sendAction();
- }
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/container-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/container-table.js
deleted file mode 100644
index 4b741b886279a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/container-table.js
+++ /dev/null
@@ -1,22 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/deploy-service.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/deploy-service.js
deleted file mode 100644
index f73f33fca08af..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/deploy-service.js
+++ /dev/null
@@ -1,179 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- viewType: 'standard',
- savedStandardTemplates: null,
- savedJsonTemplates: null,
- savedTemplateName: '',
- serviceDef: null,
- customServiceDef: '',
- serviceResp: null,
- isLoading: false,
- userName: '',
- hosts: Ember.inject.service('hosts'),
-
- actions: {
- showSaveTemplateModal() {
- this.$('#saveListModal').modal('show');
- },
-
- deployService() {
- this.set('serviceResp', null);
- if (this.get('isStandardViewType')) {
- this.sendAction("deployServiceDef", this.get('serviceDef'), this.get('userName'));
- } else {
- try {
- var parsed = JSON.parse(this.get('customServiceDef'));
- this.sendAction("deployServiceJson", parsed, this.get('userName'));
- } catch (err) {
- this.set('serviceResp', {type: 'error', message: 'Invalid JSON: ' + err.message});
- throw err;
- }
- }
- },
-
- updateViewType(type) {
- this.set('viewType', type);
- },
-
- addToSavedList() {
- this.unselectAllSavedList();
- if (this.get('isStandardViewType')) {
- this.get('savedStandardTemplates').addObject({
- name: this.get('savedTemplateName'),
- defId: this.get('serviceDef.id'),
- active: true
- });
- this.set('serviceDef.isCached', true);
- } else {
- this.get('savedJsonTemplates').addObject({
- name: this.get('savedTemplateName'),
- custom: this.get('customServiceDef'),
- active: true
- });
- }
- this.$('#saveListModal').modal('hide');
- this.set('savedTemplateName', '');
- },
-
- updateServiceDef(def) {
- this.selectActiveListItem(def);
- if (this.get('isStandardViewType')) {
- this.set('serviceDef', this.getStore().peekRecord('yarn-servicedef', def.defId));
- } else {
- this.set('customServiceDef', def.custom);
- }
- },
-
- clearConfigs() {
- this.unselectAllSavedList();
- this.set('serviceResp', null);
- if (this.get('isStandardViewType')) {
- var oldDef = this.get('serviceDef');
- var def = oldDef.createNewServiceDef();
- this.set('serviceDef', def);
- if (!oldDef.get('isCached')) {
- oldDef.deleteRecord();
- }
- } else {
- this.set('customServiceDef', '');
- }
- },
-
- removeFromSavedList(list) {
- if (list.active) {
- this.send('clearConfigs');
- }
- if (this.get('isStandardViewType')) {
- this.get('savedStandardTemplates').removeObject(list);
- } else {
- this.get('savedJsonTemplates').removeObject(list);
- }
- },
-
- clearServiceResponse() {
- this.set('serviceResp', null);
- }
- },
-
- didInsertElement() {
- var self = this;
- self.$().find('.modal').on('shown.bs.modal', function() {
- self.$().find('.modal.in').find('input.form-control:first').focus();
- });
- },
-
- selectActiveListItem(item) {
- this.unselectAllSavedList();
- Ember.set(item, 'active', true);
- },
-
- unselectAllSavedList() {
- this.get('getSavedList').forEach(function(item) {
- Ember.set(item, 'active', false);
- });
- },
-
- getSavedList: Ember.computed('viewType', function() {
- if (this.get('isStandardViewType')) {
- return this.get('savedStandardTemplates');
- } else {
- return this.get('savedJsonTemplates');
- }
- }),
-
- getStore: function() {
- return this.get('serviceDef.store');
- },
-
- isStandardViewType: Ember.computed.equal('viewType', 'standard'),
-
- isCustomViewType: Ember.computed.equal('viewType', 'custom'),
-
- isValidTemplateName: Ember.computed.notEmpty('savedTemplateName'),
-
- isUserNameGiven: Ember.computed.empty('userName'),
-
- isValidServiceDef: Ember.computed('serviceDef.name', 'serviceDef.queue',
- 'serviceDef.version', 'serviceDef.serviceComponents.[]', function () {
- return this.get('serviceDef').isValidServiceDef();
- }),
-
- isValidCustomServiceDef: Ember.computed.notEmpty('customServiceDef'),
-
- isSecurityNotEnabled: Ember.computed('isSecurityEnabled', function () {
- return this.get(`hosts.isSecurityEnabled`) === 'simple';
- }),
-
- enableSaveOrDeployBtn: Ember.computed('isValidServiceDef', 'isValidCustomServiceDef', 'viewType', 'isLoading', 'isUserNameGiven', function() {
- if (this.get('isLoading')) {
- return false;
- }
- if (this.get('isSecurityNotEnabled') && this.get('isUserNameGiven')) {
- return false;
- }
- if (this.get('isStandardViewType')) {
- return this.get('isValidServiceDef');
- } else {
- return this.get('isValidCustomServiceDef');
- }
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
deleted file mode 100644
index 03b633682f73a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/donut-chart.js
+++ /dev/null
@@ -1,209 +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.
- */
-
-import Ember from 'ember';
-import BaseChartComponent from 'yarn-ui/components/base-chart-component';
-import ColorUtils from 'yarn-ui/utils/color-utils';
-import Converter from 'yarn-ui/utils/converter';
-import {Entities} from 'yarn-ui/constants';
-
-export default BaseChartComponent.extend({
- /*
- * data = [{label="xx", value=},{...}]
- */
- renderDonutChart: function(data, title, showLabels = false,
- middleLabel = "Total", middleValue = undefined, suffix = "") {
- var g = this.chart.g;
- var layout = this.getLayout();
- this.renderTitleAndBG(g, title, layout);
-
- var total = 0;
- var allZero = true;
- for (var i = 0; i < data.length; i++) {
- total += data[i].value;
- if (data[i].value > 1e-6) {
- allZero = false;
- }
- }
-
- if (!middleValue) {
- if (this.get(Entities.Type) === Entities.Memory) {
- middleValue = Converter.memoryToSimpliedUnit(total);
- } else if (this.get(Entities.Type) === Entities.Resource) {
- middleValue = Converter.resourceToSimplifiedUnit(total, this.get(Entities.Unit));
- } else {
- middleValue = total;
- }
- }
-
- //Width and height
- var h = layout.y2 - layout.y1;
-
- // 50 is for title
- var outerRadius = (h - 50 - 2 * layout.margin) / 2;
-
- // Ratio of inner radius to outer radius
- var radiusRatio = 0.75;
- var innerRadius = outerRadius * radiusRatio;
-
- var arc = d3.svg.arc()
- .innerRadius(innerRadius)
- .outerRadius(outerRadius);
-
- var cx;
- var cy = layout.y1 + 50 + layout.margin + outerRadius;
- if (showLabels) {
- cx = layout.x1 + layout.margin + outerRadius;
- } else {
- cx = (layout.x1 + layout.x2) / 2;
- }
-
- var pie = d3.layout.pie();
- pie.sort(null);
- pie.value(function(d) {
- var v = d.value;
- // make sure it > 0
- v = Math.max(v, 1e-6);
- return v;
- });
-
- //Set up groups
- var arcs = g
- .selectAll("g.arc")
- .data(pie(data))
- .enter()
- .append("g")
- .attr("class", "arc")
- .attr("transform", "translate(" + cx + "," + cy + ")");
-
- function tweenPie(finish) {
- var start = {
- startAngle: 0,
- endAngle: 0
- };
- var i = d3.interpolate(start, finish);
- return function(d) {
- return arc(i(d));
- };
- }
-
- //Draw arc paths
- var path = arcs.append("path")
- .attr("fill", function(d, i) {
- if (d.value > 1e-6) {
- return this.colors[i];
- } else {
- return "white";
- }
- }.bind(this))
- .attr("d", arc)
- .attr("stroke", function(d, i) {
- if (allZero) {
- return this.colors[i];
- }
- }.bind(this));
- this.bindTooltip(path);
- path.on("click", function (d) {
- var data = d.data;
- if (data.link) {
- this.tooltip.remove();
- document.location.href = data.link;
- }
- }.bind(this));
-
- // Show labels
- if (showLabels) {
- var lx = layout.x1 + layout.margin + outerRadius * 2 + 30;
- var squareW = 15;
- var margin = 10;
-
- var select = g.selectAll(".rect")
- .data(data)
- .enter();
- select.append("rect")
- .attr("fill", function(d, i) {
- return this.colors[i];
- }.bind(this))
- .attr("x", lx)
- .attr("y", function(d, i) {
- return layout.y1 + 75 + (squareW + margin) * i + layout.margin;
- })
- .attr("width", squareW)
- .attr("height", squareW);
- select.append("text")
- .attr("x", lx + squareW + margin)
- .attr("y", function(d, i) {
- return layout.y1 + 80 + (squareW + margin) * i + layout.margin + squareW / 2;
- })
- .text(function(d) {
- var value = d.value;
- if (this.get("type") === "memory") {
- value = Converter.memoryToSimpliedUnit(value);
- } else if (this.get("type") === "resource") {
- value = Converter.resourceToSimplifiedUnit(value, this.get(Entities.Unit));
- }
-
- return d.label + ' = ' + value + suffix;
- }.bind(this));
- }
-
- if (middleLabel) {
- var highLightColor = this.colors[0];
- g.append("text").text(middleLabel).attr("x", cx).attr("y", cy - 10).
- attr("class", "donut-highlight-text").attr("fill", highLightColor);
- g.append("text").text(middleValue).attr("x", cx).attr("y", cy + 15).
- attr("class", "donut-highlight-sub").attr("fill", highLightColor);
- }
-
- path.transition()
- .duration(500)
- .attrTween('d', tweenPie);
- },
-
- _dataChange: Ember.observer("data", function() {
- this.chart.g.selectAll("*").remove();
- if(this.get("data")) {
- this.draw();
- }
- }),
-
- draw: function() {
- var colorTargets = this.get("colorTargets");
- if (colorTargets) {
- var colorTargetReverse = Boolean(this.get("colorTargetReverse"));
- var targets = colorTargets.split(" ");
- this.colors = ColorUtils.getColors(this.get("data").length, targets, colorTargetReverse);
- }
-
- this.renderDonutChart(this.get("data"), this.get("title"), this.get("showLabels"),
- this.get("middleLabel"), this.get("middleValue"), this.get("suffix"));
- },
-
- didInsertElement: function() {
- // When parentIdPrefix is specified, use parentidPrefix + name as new parent
- // id
- if (this.get("parentIdPrefix")) {
- var newParentId = this.get("parentIdPrefix") + this.get("id");
- this.set("parentId", newParentId);
- console.log(newParentId);
- }
-
- this.initChart();
- this.draw();
- },
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/em-table-html-cell.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/em-table-html-cell.js
deleted file mode 100644
index 56fc68a2b0b7c..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/em-table-html-cell.js
+++ /dev/null
@@ -1,23 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- content: null
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/em-table-simple-status-cell.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/em-table-simple-status-cell.js
deleted file mode 100644
index af8b605eeb0d0..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/em-table-simple-status-cell.js
+++ /dev/null
@@ -1,31 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- content: null,
-
- classNames: ["em-table-simple-status-cell"],
-
- statusName: Ember.computed("content", function () {
- var status = this.get("content");
-
- return status.toLowerCase().capitalize();
- }),
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/em-table-tooltip-text.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/em-table-tooltip-text.js
deleted file mode 100644
index f3634607fafd2..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/em-table-tooltip-text.js
+++ /dev/null
@@ -1,33 +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.
- */
-
- import Ember from 'ember';
-
-export default Ember.Component.extend({
- content: null,
-
- classNames: ["em-table-text-with-tooltip"],
-
- didRender: function() {
- this.$().parent().css("position", "static");
- },
-
- tooltipText: Ember.computed("content", function () {
- return this.get("content");
- }),
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/fileconfig-viewer-dialog.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/fileconfig-viewer-dialog.js
deleted file mode 100644
index d4912768b18ab..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/fileconfig-viewer-dialog.js
+++ /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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- dialogId: "fileconfig_viewer_dialog",
- title: "File Configuration Properties",
- props: null,
- customProps: Ember.computed('props', function() {
- var custom = [];
- var props = this.get('props');
- for (var pro in props) {
- custom.push({
- name: pro,
- value: props[pro]
- });
- }
- return custom;
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/gpu-donut-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/gpu-donut-chart.js
deleted file mode 100644
index fa5ca8ac1dce2..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/gpu-donut-chart.js
+++ /dev/null
@@ -1,66 +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.
- */
-
-import DonutChart from 'yarn-ui/components/donut-chart';
-import ColorUtils from 'yarn-ui/utils/color-utils';
-
-export default DonutChart.extend({
- draw: function() {
- // Construct data
- var data = [];
- if (this.get("gpu-render-type") === "gpu-memory") {
- data.push({
- label: "Used",
- value: parseFloat(this.get("gpuInfo").gpuMemoryUsage.usedMemoryMiB),
- });
- data.push({
- label: "Available",
- value: parseFloat(this.get("gpuInfo").gpuMemoryUsage.availMemoryMiB)
- });
- } else if (this.get("gpu-render-type") === "gpu-utilization") {
- var utilization = parseFloat(this.get("gpuInfo").gpuUtilizations.overallGpuUtilization);
- data.push({
- label: "Utilized",
- value: utilization,
- });
- data.push({
- label: "Available",
- value: 100 - utilization
- });
- }
-
- var colorTargets = this.get("colorTargets");
- if (colorTargets) {
- var colorTargetReverse = Boolean(this.get("colorTargetReverse"));
- var targets = colorTargets.split(" ");
- this.colors = ColorUtils.getColors(data.length, targets, colorTargetReverse);
- }
-
- this.renderDonutChart(data, this.get("title"), this.get("showLabels"),
- this.get("middleLabel"), this.get("middleValue"), this.get("suffix"));
- },
-
- didInsertElement: function() {
- // ParentId includes minorNumber
- var newParentId = this.get("parentId") + this.get("gpuInfo").minorNumber;
- this.set("parentId", newParentId);
-
- this.initChart();
- this.draw();
- },
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/info-tooltip.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/info-tooltip.js
deleted file mode 100644
index 605b611f0b70a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/info-tooltip.js
+++ /dev/null
@@ -1,44 +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.
- */
-
-import Ember from 'ember';
-import InfoSeeder from 'yarn-ui/utils/info-seeder';
-
-export default Ember.Component.extend({
- classNames: ['tooltip', 'info-tooltip'],
- elementId: 'info_tooltip_wrapper',
-
- didInsertElement() {
- var $tooltip = Ember.$('#info_tooltip_wrapper');
- Ember.$('body').on('mouseenter', '.info-icon', function() {
- var $elem = Ember.$(this);
- var info = InfoSeeder[$elem.data('info')];
- var offset = $elem.offset();
- $tooltip.show();
- $tooltip.find("#tooltip_content").text(info);
- $tooltip.offset({top: offset.top + 20, left: offset.left - 10});
- }).on('mouseleave', '.info-icon', function() {
- $tooltip.find("#tooltip_content").text('');
- $tooltip.hide();
- });
- },
-
- WillDestroyElement() {
- Ember.$('body').off('hover');
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/item-selector.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/item-selector.js
deleted file mode 100644
index 235e4386e4577..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/item-selector.js
+++ /dev/null
@@ -1,39 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- didInsertElement: function() {
- $(".js-example-basic-single").select2(
- {
- width: '100%',
- placeholder: "Select a queue"
- });
- var elementId = this.get("element-id");
- var prefix = this.get("prefix");
-
- var element = d3.select("#" + elementId);
-
- if (element) {
- this.get("model").forEach(function(o) {
- element.append("option").attr("value", o.get("name")).text(prefix + o.get("name"));
- });
- }
- }
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/metrics-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/metrics-table.js
deleted file mode 100644
index 62b2fc5964cff..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/metrics-table.js
+++ /dev/null
@@ -1,24 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- metrics: null,
- type: ''
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/node-menu-panel.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/node-menu-panel.js
deleted file mode 100644
index 31457bed20a96..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/node-menu-panel.js
+++ /dev/null
@@ -1,25 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- encodedAddr : Ember.computed("nodeAddr", function(){
- return encodeURIComponent(this.get('nodeAddr'));
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
deleted file mode 100644
index 7eac266505aca..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/nodes-heatmap.js
+++ /dev/null
@@ -1,317 +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.
- */
-
-import BaseChartComponent from 'yarn-ui/components/base-chart-component';
-
-export default BaseChartComponent.extend({
- CELL_WIDTH: 250,
- SAMPLE_CELL_WIDTH: 100,
- SAMPLE_HEIGHT: 30,
- CELL_HEIGHT: 30,
- CELL_MARGIN: 2,
- RACK_MARGIN: 20,
- filter: "",
- selectedCategory: 0,
- memoryLabel: "Memory",
- cpuLabel: "VCores",
- containersLabel: "Containers",
- totalContainers: 0,
-
- bindTP: function(element, cell) {
- var currentToolTip = this.tooltip;
- element.on("mouseover", function() {
- currentToolTip
- .style("left", (d3.event.pageX) + "px")
- .style("top", (d3.event.pageY - 28) + "px");
- cell.style("opacity", 1.0);
- }.bind(this))
- .on("mousemove", function() {
- // Handle pie chart case
- var text = cell.attr("tooltiptext");
- currentToolTip
- .style("background", "black")
- .style("opacity", 0.7);
- currentToolTip
- .html(text)
- .style('font-size', '12px')
- .style('color', 'white')
- .style('font-weight', '400');
- currentToolTip
- .style("left", (d3.event.pageX) + "px")
- .style("top", (d3.event.pageY - 28) + "px");
- }.bind(this))
- .on("mouseout", function() {
- currentToolTip.style("opacity", 0);
- cell.style("opacity", 0.8);
- }.bind(this));
- },
-
- bindSelectCategory: function(element, i) {
- element.on("click", function() {
- if (this.selectedCategory === i) {
- // Remove selection for second click
- this.selectedCategory = 0;
- } else {
- this.selectedCategory = i;
- }
- this.didInsertElement();
- }.bind(this));
- },
-
- isNodeSelected: function(node) {
- if (this.filter) {
- var rack = node.get("rack");
- var host = node.get("nodeHostName");
- if (!rack.includes(this.filter) && !host.includes(this.filter)) {
- return false;
- }
- }
-
- if (this.selectedCategory === 0) {
- return true;
- }
-
- var usage = this.calcUsage(node);
- var lowerLimit = (this.selectedCategory - 1) * 0.2;
- var upperLimit = this.selectedCategory * 0.2;
- if (lowerLimit <= usage && usage <= upperLimit) {
- return true;
- }
- return false;
- },
-
- // data:
- // [{label=label1, value=value1}, ...]
- // ...
- renderCells: function (model, title) {
- var selectedOption = d3.select("select").property("value");
- var data = [];
- model.forEach(function (o) {
- data.push(o);
- });
-
- this.chart.g.remove();
- this.chart.g = this.chart.svg.append("g");
- var g = this.chart.g;
- var layout = this.getLayout();
- layout.margin = 50;
-
- let racks = new Set();
- for (var i = 0; i < data.length; i++) {
- racks.add(data[i].get("rack"));
- }
-
- let racksArray = [];
- racks.forEach(v => racksArray.push(v));
-
- var xOffset = layout.margin;
- var yOffset = layout.margin * 3;
-
- var gradientStartColor = "#60cea5";
- var gradientEndColor = "#ffbc0b";
-
- var colorFunc = d3.interpolateRgb(d3.rgb(gradientStartColor), d3.rgb(gradientEndColor));
-
- var sampleXOffset = (layout.x2 - layout.x1) / 2 - 2.5 * this.SAMPLE_CELL_WIDTH -
- 2 * this.CELL_MARGIN;
- var sampleYOffset = layout.margin * 2;
- var text;
-
- for (i = 1; i <= 5; i++) {
- var ratio = i * 0.2 - 0.1;
-
- var rect = g.append("rect")
- .attr("x", sampleXOffset)
- .attr("y", sampleYOffset)
- .attr("fill", this.selectedCategory === i ? "#26bbf0" : colorFunc(ratio))
- .attr("width", this.SAMPLE_CELL_WIDTH)
- .attr("height", this.SAMPLE_HEIGHT)
- .attr("class", "hyperlink");
- this.bindSelectCategory(rect, i);
- text = g.append("text")
- .text("" + (ratio * 100).toFixed(1) + "% Used")
- .attr("y", sampleYOffset + this.SAMPLE_HEIGHT / 2 + 5)
- .attr("x", sampleXOffset + this.SAMPLE_CELL_WIDTH / 2)
- .attr("class", "heatmap-cell hyperlink");
- this.bindSelectCategory(text, i);
- sampleXOffset += this.CELL_MARGIN + this.SAMPLE_CELL_WIDTH;
- }
-
- if (this.selectedCategory !== 0) {
- text = g.append("text")
- .text("Clear")
- .attr("y", sampleYOffset + this.SAMPLE_HEIGHT / 2 + 5)
- .attr("x", sampleXOffset + 20)
- .attr("class", "heatmap-clear hyperlink");
- this.bindSelectCategory(text, 0);
- }
-
- var chartXOffset = -1;
-
- this.totalContainers = 0;
- for (i = 0; i < racksArray.length; i++) {
- text = g.append("text")
- .text(racksArray[i])
- .attr("y", yOffset + this.CELL_HEIGHT / 2 + 5)
- .attr("x", layout.margin)
- .attr("class", "heatmap-rack");
-
- if (-1 === chartXOffset) {
- chartXOffset = layout.margin + text.node().getComputedTextLength() + 30;
- }
-
- xOffset = chartXOffset;
-
- for (var j = 0; j < data.length; j++) {
- var rack = data[j].get("rack");
-
- if (rack === racksArray[i]) {
- this.totalContainers += data[j].get("numContainers");
- this.addNode(g, xOffset, yOffset, colorFunc, data[j]);
- xOffset += this.CELL_MARGIN + this.CELL_WIDTH;
- if (xOffset + this.CELL_MARGIN + this.CELL_WIDTH >= layout.x2 -
- layout.margin) {
- xOffset = chartXOffset;
- yOffset = yOffset + this.CELL_MARGIN + this.CELL_HEIGHT;
- }
-
- }
- }
-
- while (xOffset > chartXOffset && xOffset + this.CELL_MARGIN +
- this.CELL_WIDTH < layout.x2 - layout.margin) {
- this.addPlaceholderNode(g, xOffset, yOffset);
- xOffset += this.CELL_MARGIN + this.CELL_WIDTH;
- }
-
- if (xOffset !== chartXOffset) {
- xOffset = chartXOffset;
- yOffset += this.CELL_MARGIN + this.CELL_HEIGHT;
- }
- yOffset += this.RACK_MARGIN;
- }
-
- layout.y2 = yOffset + layout.margin;
- this.adjustMaxHeight(layout.y2);
- this.renderTitleAndBG(g, title + selectedOption + ")" , layout, false);
- },
-
- addNode: function (g, xOffset, yOffset, colorFunc, data) {
- var rect = g.append("rect")
- .attr("y", yOffset)
- .attr("x", xOffset)
- .attr("height", this.CELL_HEIGHT)
- .attr("fill", colorFunc(this.calcUsage(data)))
- .attr("width", this.CELL_WIDTH)
- .attr("tooltiptext", data.get("toolTipText") + this.getToolTipText(data));
-
- if (this.isNodeSelected(data)) {
- rect.style("opacity", 0.8);
- this.bindTP(rect, rect);
- } else {
- rect.style("opacity", 0.8);
- rect.attr("fill", "DimGray");
- }
- var node_id = data.get("id"),
- node_addr = encodeURIComponent(data.get("nodeHTTPAddress")),
- href = `#/yarn-node/${node_id}/${node_addr}/info`;
- var nodeHostName = data.get("nodeHostName");
- var a = g.append("a")
- .attr("href", href);
- a.append("text")
- .text(nodeHostName.length > 30 ? nodeHostName.substr(0, 30) + '...' : nodeHostName)
- .attr("y", yOffset + this.CELL_HEIGHT / 2 + 5)
- .attr("x", nodeHostName.length > 30 ? xOffset + 10 : xOffset + this.CELL_WIDTH / 2)
- .style("text-anchor", nodeHostName.length > 30 ? "start" : "middle")
- .attr("class", this.isNodeSelected(data) ? "heatmap-cell" : "heatmap-cell-notselected");
- if (this.isNodeSelected(data)) {
- this.bindTP(a, rect);
- }
- },
-
- addPlaceholderNode: function(g, xOffset, yOffset) {
- g.append("rect")
- .attr("y", yOffset)
- .attr("x", xOffset)
- .attr("height", this.CELL_HEIGHT)
- .attr("fill", "grey")
- .attr("width", this.CELL_WIDTH)
- .style("opacity", 0.20);
- },
-
- draw: function() {
- this.initChart(true);
- this.renderCells(this.get("model"), this.get("title"), this.get("textWidth"));
- },
-
- didInsertElement: function () {
- var self = this;
- var optionsData = [this.memoryLabel, this.cpuLabel, this.containersLabel];
- d3.select("#heatmap-select")
- .on('change', function() {
- self.renderCells(self.get("model"), self.get("title"), self.get("textWidth"));
- })
- .selectAll('option')
- .data(optionsData).enter()
- .append('option')
- .text(function (d) { return d; });
-
- this.draw();
- },
-
- actions: {
- applyFilter: function(event) {
- this.filter = event.srcElement.value;
- this.selectedCategory = 0;
- this.didInsertElement();
- }
- },
-
- calcUsage: function(data) {
- var selectedOption = d3.select('select').property("value");
- if (selectedOption === this.memoryLabel) {
- return data.get("usedMemoryMB") /
- (data.get("usedMemoryMB") + data.get("availMemoryMB"));
- }
- else if (selectedOption === this.cpuLabel) {
- return data.get("usedVirtualCores") /
- (data.get("usedVirtualCores") + data.get("availableVirtualCores"));
- }
- else if (selectedOption === this.containersLabel) {
- var totalContainers = this.totalContainers;
- if (totalContainers === 0) { return 0; }
- return data.get("numContainers") / totalContainers;
- }
- },
-
- getToolTipText: function(data) {
- var selectedOption = d3.select('select').property("value");
- if (selectedOption === this.memoryLabel) {
- return "
Used Memory: " + Math.round(data.get("usedMemoryMB")) + " MB
" +
- "
Available Memory: " + Math.round(data.get("availMemoryMB")) + " MB
Total Containers: " + this.totalContainers + " Containers
";
- }
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/per-app-memusage-by-nodes-stacked-barchart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/per-app-memusage-by-nodes-stacked-barchart.js
deleted file mode 100644
index ef81c05dca994..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/per-app-memusage-by-nodes-stacked-barchart.js
+++ /dev/null
@@ -1,86 +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.
- */
-
-import StackedBarchart from 'yarn-ui/components/stacked-barchart';
-import Converter from 'yarn-ui/utils/converter';
-
-export default StackedBarchart.extend({
- getDataForRender: function(containers, nodes) {
- var arr = [];
- var nodeToResources = {};
- nodes.forEach(function(n) {
- nodeToResources[n.id] =
- {
- used: Number(n.get("usedMemoryMB")),
- avail: Number(n.get("availMemoryMB"))
- };
- });
-
- containers.forEach(function(c) {
- res = nodeToResources[c.get("assignedNodeId")];
- if (res) {
- if (!res.usedByTheApp) {
- res.usedByTheApp = 0;
- }
- res.usedByTheApp += Number(c.get("allocatedMB"));
- }
- });
-
- for (var nodeId in nodeToResources) {
- var res = nodeToResources[nodeId];
-
- var subArr = [];
- var value = res.usedByTheApp ? res.usedByTheApp : 0;
- subArr.push({
- value: value,
- bindText: "This app uses " + Converter.memoryToSimpliedUnit(value) + " on node=" + nodeId,
- });
-
- value = res.used - value;
- value = Math.max(value, 0);
- subArr.push({
- value: value,
- bindText: "Other applications use " + Converter.memoryToSimpliedUnit(value) + " on node=" + nodeId,
- });
-
- subArr.push({
- value: res.avail,
- bindText: Converter.memoryToSimpliedUnit(res.avail) + " memory is available on node=" + nodeId
- });
-
- arr.push(subArr);
- }
-
- return arr;
- },
-
- didInsertElement: function() {
- this.initChart(true);
-
- this.colors = ["lightsalmon", "Grey", "mediumaquamarine"];
-
- var containers = this.get("rmContainers");
- var nodes = this.get("nodes");
-
- var data = this.getDataForRender(containers, nodes);
-
- this.show(
- data, this.get("title"), ["Used by this app", "Used by other apps", "Available"]
- );
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/per-app-ncontainers-by-nodes-stacked-barchart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/per-app-ncontainers-by-nodes-stacked-barchart.js
deleted file mode 100644
index 27822d2b357c5..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/per-app-ncontainers-by-nodes-stacked-barchart.js
+++ /dev/null
@@ -1,65 +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.
- */
-
-import StackedBarchart from 'yarn-ui/components/stacked-barchart';
-
-export default StackedBarchart.extend({
- getDataForRender: function(containers, nodes) {
- var arr = [];
- var nodeToContainers = {};
- nodes.forEach(function(n) {
- nodeToContainers[n.id] = 0;
- });
-
- containers.forEach(function(c) {
- var nodeId = c.get("assignedNodeId");
- var n = nodeToContainers[nodeId];
- if (undefined !== n) {
- nodeToContainers[nodeId] += 1;
- }
- });
-
- for (var nodeId in nodeToContainers) {
- var n = nodeToContainers[nodeId];
-
- var subArr = [];
- subArr.push({
- value: n,
- bindText: "This app has " + n + " containers running on node=" + nodeId
- });
-
- arr.push(subArr);
- }
-
- return arr;
- },
-
- didInsertElement: function() {
- this.initChart(true);
-
- this.colors = ["Orange", "Grey", "Gainsboro"];
-
- var containers = this.get("rmContainers");
- var nodes = this.get("nodes");
-
- var data = this.getDataForRender(containers, nodes);
-
- this.show(
- data, this.get("title"), ["Running containers from this app"]);
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/per-app-vcoreusage-by-nodes-stacked-barchart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/per-app-vcoreusage-by-nodes-stacked-barchart.js
deleted file mode 100644
index acb6f6e059909..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/per-app-vcoreusage-by-nodes-stacked-barchart.js
+++ /dev/null
@@ -1,85 +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.
- */
-
-import StackedBarchart from 'yarn-ui/components/stacked-barchart';
-
-export default StackedBarchart.extend({
- getDataForRender: function(containers, nodes) {
- var arr = [];
- var nodeToResources = {};
- nodes.forEach(function(n) {
- nodeToResources[n.id] =
- {
- used: Number(n.get("usedVirtualCores")),
- avail: Number(n.get("availableVirtualCores"))
- };
- });
-
- containers.forEach(function(c) {
- res = nodeToResources[c.get("assignedNodeId")];
- if (res) {
- if (!res.usedByTheApp) {
- res.usedByTheApp = 0;
- }
- res.usedByTheApp += Number(c.get("allocatedVCores"));
- }
- });
-
- for (var nodeId in nodeToResources) {
- var res = nodeToResources[nodeId];
-
- var subArr = [];
- var value = res.usedByTheApp ? res.usedByTheApp : 0;
- subArr.push({
- value: value,
- bindText: "This app uses " + value + " vcores on node=" + nodeId,
- });
-
- value = res.used - value;
- value = Math.max(value, 0);
- subArr.push({
- value: value,
- bindText: "Other applications use " + value + " vcores on node=" + nodeId,
- });
-
- subArr.push({
- value: res.avail,
- bindText: res.avail + (res.avail > 1 ? " vcores are" : " vcore is") + " available on node=" + nodeId
- });
-
- arr.push(subArr);
- }
-
- return arr;
- },
-
- didInsertElement: function() {
- this.initChart(true);
-
- this.colors = ["lightsalmon", "Grey", "mediumaquamarine"];
-
- var containers = this.get("rmContainers");
- var nodes = this.get("nodes");
-
- var data = this.getDataForRender(containers, nodes);
-
- this.show(
- data, this.get("title"), ["Used by this app", "Used by other apps", "Available"]
- );
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-configuration-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-configuration-table.js
deleted file mode 100644
index 4b741b886279a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-configuration-table.js
+++ /dev/null
@@ -1,22 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js
deleted file mode 100644
index 2cecefb772016..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js
+++ /dev/null
@@ -1,42 +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.
- */
-
-import Ember from "ember";
-
-export default Ember.Component.extend(Ember.TargetActionSupport,{
- actions: {
- filterQueuesByPartition(filter) {
- this.set("filteredPartition", filter);
- this.sendAction("setFilter", filter);
- }
- },
- didInsertElement: function() {
- $(".js-filter-queue-by-labels").select2({
- width: "350px",
- multiple: false
- });
-
- $(".js-filter-queue-by-labels").on("select2:select", e => {
- this.triggerAction({
- action: "filterQueuesByPartition",
- target: this,
- actionContext: e.params.data.text
- });
- });
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-usage-donut-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-usage-donut-chart.js
deleted file mode 100644
index c939aaf80ad53..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-usage-donut-chart.js
+++ /dev/null
@@ -1,67 +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.
- */
-
-import BaseUsageDonutChart from 'yarn-ui/components/base-usage-donut-chart';
-import ColorUtils from 'yarn-ui/utils/color-utils';
-import HrefAddressUtils from 'yarn-ui/utils/href-address-utils';
-
-export default BaseUsageDonutChart.extend({
- colors: d3.scale.category20().range(),
-
- draw: function() {
- var usageByQueues = [];
- var avail = 100;
-
- this.get("data").forEach(function (queue) {
- var v = queue.get("absUsedCapacity");
-
- if (queue.get("isLeafQueue")) {
- if (v > 1e-2) {
- usageByQueues.push({
- label: queue.get("id"),
- link: HrefAddressUtils.getQueueLink(queue.get("id")),
- value: v.toFixed(2)
- });
-
- avail = avail - v;
- }
- }
- });
-
- usageByQueues.sort(function(a, b) {
- return b.value - a.value;
- });
-
- usageByQueues = this.mergeLongTails(usageByQueues, 8);
-
- usageByQueues.push({
- label: "Available",
- value: avail.toFixed(4)
- });
-
- this.colors = ColorUtils.getColors(usageByQueues.length, ["others", "good"], true);
-
- this.renderDonutChart(usageByQueues, this.get("title"), this.get("showLabels"),
- this.get("middleLabel"), "100%", "%");
- },
-
- didInsertElement: function() {
- this.initChart();
- this.draw();
- },
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-view.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-view.js
deleted file mode 100644
index f5fb68d1c83bf..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-view.js
+++ /dev/null
@@ -1,288 +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.
- */
-
-import Ember from 'ember';
-import ChartUtilsMixin from 'yarn-ui/mixins/charts-utils';
-
-export default Ember.Component.extend(ChartUtilsMixin, {
- queues: {
- data: undefined,
- foldedQueues: {},
- selectedQueueCircle: undefined,
- maxDepth: -1,
- },
-
- queueColors: d3.scale.category20().range(),
-
- renderQueue: function (now, depth, sequence) {
- if (depth > this.queues.maxDepth) {
- this.queues.maxDepth = depth;
- }
-
- var cx = 20 + depth * 30;
- var cy = 20 + sequence * 30;
- var name = now.get("name");
-
- var g = this.queues.dataGroup.append("g")
- .attr("id", "queue-" + name + "-g");
-
- var folded = this.queues.foldedQueues[name];
- var isParentQueue = false;
-
- // render its children
- var children = [];
- var childrenNames = now.get("children");
- if (childrenNames) {
- childrenNames.forEach(function (name) {
- isParentQueue = true;
- var child = this.queues.data[name];
- if (child) {
- children.push(child);
- }
- }.bind(this));
- }
- if (folded) {
- children = [];
- }
- var linefunction = d3.svg.line()
- .interpolate("basis")
- .x(function (d) {
- return d.x;
- })
- .y(function (d) {
- return d.y;
- });
-
- for (var i = 0; i < children.length; i++) {
- sequence = sequence + 1;
- // Get center of children queue
- var cc = this.renderQueue(children[i],
- depth + 1, sequence);
- g.append("path")
- .attr("class", "queue")
- .attr("d", linefunction([{
- x: cx,
- y: cy
- }, {
- x: cc.x - 20,
- y: cc.y
- }, cc]));
- }
-
- var circle = g.append("circle")
- .attr("cx", cx)
- .attr("cy", cy)
- .attr("class", "queue");
-
- circle.on('mouseover', function () {
- }.bind(this));
- circle.on('mouseout', function () {
- if (circle !== this.queues.selectedQueueCircle) {
- circle.style("fill", this.queueColors[0]);
- }
- }.bind(this));
- circle.on('click', function () {
- circle.style("fill", this.queueColors[2]);
- var pre = this.queues.selectedQueueCircle;
- this.queues.selectedQueueCircle = circle;
- if (pre) {
- pre.on('mouseout')();
- }
- this.renderCharts(name);
- }.bind(this));
- circle.on('dblclick', function () {
- if (!isParentQueue) {
- return;
- }
-
- if (this.queues.foldedQueues[name]) {
- delete this.queues.foldedQueues[name];
- } else {
- this.queues.foldedQueues[name] = now;
- }
- this.renderQueues();
- }.bind(this));
-
- var text = name;
- if (folded) {
- text = name + " (+)";
- }
-
- // print queue's name
- g.append("text")
- .attr("x", cx + 30)
- .attr("y", cy + 5)
- .text(text)
- .attr("class", "queue");
-
- return {
- x: cx,
- y: cy
- };
- },
-
- renderQueues: function () {
- if (this.queues.dataGroup) {
- this.queues.dataGroup.remove();
- }
- // render queues
- this.queues.dataGroup = this.canvas.svg.append("g")
- .attr("id", "queues-g");
-
- if (this.queues.data) {
- this.renderQueue(this.queues.data['root'], 0, 0);
-
- }
- },
-
- draw: function () {
- this.queues.data = {};
- this.get("model")
- .forEach(function (o) {
- this.queues.data[o.id] = o;
- }.bind(this));
-
- // get w/h of the svg
- var bbox = d3.select("#main-container")
- .node()
- .getBoundingClientRect();
- this.canvas.w = bbox.width;
- this.canvas.h = Math.max(Object.keys(this.queues.data)
- .length * 35, 1500);
-
- this.canvas.svg = d3.select("#main-container")
- .append("svg")
- .attr("width", this.canvas.w)
- .attr("height", this.canvas.h)
- .attr("id", "main-svg");
-
- this.renderBackground();
-
- this.renderQueues();
- this.renderCharts("root");
- },
-
- didInsertElement: function () {
- this.draw();
- },
-
- /*
- * data = [{label="xx", value=},{...}]
- */
- renderTable: function (data) {
- d3.select("#main-svg")
- .append('table')
- .selectAll('tr')
- .data(data)
- .enter()
- .append('tr')
- .selectAll('td')
- .data(function (d) {
- return d;
- })
- .enter()
- .append('td')
- .text(function (d) {
- return d;
- });
- },
-
- renderQueueCapacities: function (queue, layout) {
- // Render bar chart
- this.renderCells(this.charts.g, [{
- label: "Cap",
- value: queue.get("capacity")
- }, {
- label: "MaxCap",
- value: queue.get("maxCapacity")
- }, {
- label: "UsedCap",
- value: queue.get("usedCapacity")
- }], "Queue Capacities", layout, 60);
- },
-
- renderChildrenCapacities: function (queue, layout) {
- var data = [];
- var children = queue.get("children");
- if (children) {
- for (var i = 0; i < children.length; i++) {
- var child = this.queues.data[children[i]];
- data.push({
- label: child.get("name"),
- value: child.get("capacity")
- });
- }
- }
-
- this.renderDonutChart(this.charts.g, data, "Children Capacities", layout, true);
- },
-
- renderChildrenUsedCapacities: function (queue, layout) {
- var data = [];
- var children = queue.get("children");
- if (children) {
- for (var i = 0; i < children.length; i++) {
- var child = this.queues.data[children[i]];
- data.push({
- label: child.get("name"),
- value: child.get("usedCapacity")
- });
- }
- }
-
- this.renderDonutChart(this.charts.g, data, "Children Used Capacities", layout, true);
- },
-
- renderLeafQueueUsedCapacities: function (layout) {
- var leafQueueUsedCaps = [];
- for (var queueName in this.queues.data) {
- var q = this.queues.data[queueName];
- if ((!q.get("children")) || q.get("children")
- .length === 0) {
- // it's a leafqueue
- leafQueueUsedCaps.push({
- label: q.get("name"),
- value: q.get("usedCapacity")
- });
- }
- }
-
- this.renderDonutChart(this.charts.g, leafQueueUsedCaps, "LeafQueues Used Capacities",
- layout, true);
- },
-
- renderCharts: function (queueName) {
- this.charts.leftBannerLen = this.queues.maxDepth * 30 + 100;
- this.initCharts();
-
- var queue = this.queues.data[queueName];
- var idx = 0;
-
- if (queue.get("name") === "root") {
- this.renderLeafQueueUsedCapacities(this.getLayout(idx++));
- }
- if (queue.get("name") !== "root") {
- this.renderQueueCapacities(queue, this.getLayout(idx++));
- }
- if (queue.get("children") && queue.get("children")
- .length > 0) {
- this.renderChildrenCapacities(queue, this.getLayout(idx++));
- this.renderChildrenUsedCapacities(queue, this.getLayout(idx++));
- }
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/service-component-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/service-component-table.js
deleted file mode 100644
index 23c2cfb8de1f6..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/service-component-table.js
+++ /dev/null
@@ -1,56 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- serviceDef: null,
- currentComponent: null,
- duplicateNameError: false,
-
- actions: {
- showAddComponentModal() {
- var newComp = this.get('serviceDef').createNewServiceComponent();
- this.set('currentComponent', newComp);
- this.set('duplicateNameError', false);
- this.$('#addComponentModal').modal('show');
- },
-
- addNewComponent() {
- this.set('duplicateNameError', false);
- if (this.isCurrentNameDuplicate()) {
- this.set('duplicateNameError', true);
- return;
- }
- this.get('serviceDef.serviceComponents').addObject(this.get('currentComponent'));
- this.$('#addComponentModal').modal('hide');
- },
-
- removeComponent(component) {
- this.get('serviceDef.serviceComponents').removeObject(component);
- }
- },
-
- isCurrentNameDuplicate() {
- var currName = this.get('currentComponent.name');
- var item = this.get('serviceDef.serviceComponents').findBy('name', currName);
- return !Ember.isNone(item);
- },
-
- isValidCurrentComponent: Ember.computed.and('currentComponent', 'currentComponent.name', 'currentComponent.cpus', 'currentComponent.memory', 'currentComponent.numOfContainers', 'currentComponent.launchCommand')
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/service-config-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/service-config-table.js
deleted file mode 100644
index b0a78dd27c1ad..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/service-config-table.js
+++ /dev/null
@@ -1,89 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- serviceDef: null,
- currentConfig: null,
- serviceConfigJson: '',
-
- actions: {
- showNewConfigurationModal() {
- var newConfig = this.get('serviceDef').createNewServiceConfig();
- this.set('currentConfig', newConfig);
- this.$('#addConfigurationModal').modal('show');
- if (this.get('isNonEmptyComponents') && this.get('currentConfig.componentName') === '') {
- this.set('currentConfig.componentName', this.get('componentNames.firstObject'));
- }
- },
-
- removeConfiguration(config) {
- this.get('serviceDef.serviceConfigs').removeObject(config);
- },
-
- configTypeChanged(type) {
- this.set('currentConfig.type', type);
- if (type === 'quicklink') {
- this.set('currentConfig.scope', 'service');
- this.set('currentConfig.componentName', '');
- }
- },
-
- addNewConfiguration() {
- this.get('serviceDef.serviceConfigs').addObject(this.get('currentConfig'));
- this.$('#addConfigurationModal').modal('hide');
- },
-
- showServiceConfigUploadModal() {
- this.set('serviceConfigJson', '');
- this.$("#service_config_upload_modal").modal('show');
- },
-
- uploadServiceConfig(json) {
- this.get('serviceDef').convertJsonServiceConfigs(json);
- this.$("#service_config_upload_modal").modal('hide');
- },
-
- configScopeChanged(scope) {
- this.set('currentConfig.scope', scope);
- },
-
- scopeComponentChanged(name) {
- this.set('currentConfig.componentName', name);
- }
- },
-
- isNonEmptyComponents: Ember.computed('serviceDef.serviceComponents.length', function() {
- return this.get('serviceDef.serviceComponents.length') > 0;
- }),
-
- isNotQuicklink: Ember.computed('currentConfig.type', function() {
- return this.get('currentConfig.type') !== "quicklink";
- }),
-
- componentNames: Ember.computed('serviceDef.serviceComponents.[]', function() {
- var names = [];
- this.get('serviceDef.serviceComponents').forEach(function(cmp) {
- names.push(cmp.get('name'));
- });
- return names;
- }),
-
- isValidCurrentConfig: Ember.computed.and('currentConfig', 'currentConfig.name', 'currentConfig.value')
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/service-fileconfig-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/service-fileconfig-table.js
deleted file mode 100644
index 7c06152813e04..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/service-fileconfig-table.js
+++ /dev/null
@@ -1,112 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- serviceDef: null,
- currentFileConfig: null,
- fileConfigJson: '',
- fileConfigProps: '',
- propertyViewer: null,
- parseError: '',
-
- actions: {
- showNewConfigFileModal() {
- var newFile = this.get('serviceDef').createNewFileConfig();
- this.set('currentFileConfig', newFile);
- this.set('fileConfigProps', '');
- this.set('parseError', '');
- this.$('#addFileConfigModal').modal('show');
- if (this.get('isNonEmptyComponents') && this.get('currentFileConfig.componentName') === '') {
- this.set('currentFileConfig.componentName', this.get('componentNames.firstObject'));
- }
- },
-
- removeFileConfiguration(file) {
- this.get('serviceDef.fileConfigs').removeObject(file);
- },
-
- addNewFileConfig() {
- this.set('parseError', '');
- var props = this.get('fileConfigProps');
- if (props) {
- try {
- var parsed = JSON.parse(props);
- this.set('currentFileConfig.props', parsed);
- } catch (err) {
- this.set('parseError', `Invalid JSON: ${err.message}`);
- throw err;
- }
- }
- this.get('serviceDef.fileConfigs').addObject(this.get('currentFileConfig'));
- this.$('#addFileConfigModal').modal('hide');
- },
-
- showFileConfigUploadModal() {
- this.set('fileConfigJson', '');
- this.$("#service_file_config_upload_modal").modal('show');
- },
-
- uploadFileConfig(json) {
- this.get('serviceDef').convertJsonFileConfigs(json);
- this.$("#service_file_config_upload_modal").modal('hide');
- },
-
- configScopeChanged(scope) {
- this.set('currentFileConfig.scope', scope);
- },
-
- scopeComponentChanged(name) {
- this.set('currentFileConfig.componentName', name);
- },
-
- configTypeChanged(type) {
- this.set('currentFileConfig.type', type);
- if (type === "TEMPLATE") {
- this.set('currentFileConfig.props', null);
- this.set('fileConfigProps', '');
- }
- },
-
- showFileConfigPropertyViewer(props) {
- this.set('propertyViewer', props);
- this.$("#file_config_properties_viewer").modal('show');
- }
- },
-
- isNonEmptyComponents: Ember.computed('serviceDef.serviceComponents.length', function() {
- return this.get('serviceDef.serviceComponents.length') > 0;
- }),
-
- componentNames: Ember.computed('serviceDef.serviceComponents.[]', function() {
- var names = [];
- this.get('serviceDef.serviceComponents').forEach(function(cmp) {
- names.push(cmp.get('name'));
- });
- return names;
- }),
-
- isValidCurrentFileConfig: Ember.computed('currentFileConfig', 'currentFileConfig.srcFile', 'currentFileConfig.destFile', 'fileConfigProps', function() {
- return this.get('currentFileConfig') && this.get('currentFileConfig.destFile') && (this.get('currentFileConfig.srcFile') || this.get('fileConfigProps'));
- }),
-
- isConfigTypeHadoopXml: Ember.computed('currentFileConfig.type', function() {
- return this.get('currentFileConfig.type') === 'HADOOP_XML';
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/simple-bar-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/simple-bar-chart.js
deleted file mode 100644
index 8527e91867fa6..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/simple-bar-chart.js
+++ /dev/null
@@ -1,206 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- data: [],
- xAxisTickFormatter: null,
- yAxisTickFormatter: null,
- xAxisText: '',
- yAxisText: '',
- enableTooltip: true,
- onBarChartClickCallback: Ember.K,
- hideTootlipOnBarChartClick: true,
-
- initChart() {
- this.height = 400;
- this.barWidth = 30;
- this.width = Math.max(500, 40 * this.data.length);
- },
-
- drawChart() {
- var margin = {top: 20, right: 20, bottom: 100, left: 100},
- axisLabelPadding = 10,
- width = this.width - margin.left - margin.right - axisLabelPadding,
- height = this.height - margin.top - margin.bottom - axisLabelPadding,
- xAxisText = this.xAxisText? this.xAxisText : '',
- yAxisText = this.yAxisText? this.yAxisText : '',
- data = this.data,
- self = this;
-
- var xScale = d3.scale.ordinal().rangeRoundBands([0, width], 0.1);
- var yScale = d3.scale.linear().range([height, 0]);
-
- var xAxis = d3.svg.axis()
- .scale(xScale)
- .orient("bottom")
- .tickFormat(function(tick) {
- if (self.isFunction(self.xAxisTickFormatter)) {
- return self.xAxisTickFormatter(tick);
- } else {
- return tick;
- }
- });
-
- var yAxis = d3.svg.axis()
- .scale(yScale)
- .orient("left")
- .tickFormat(function(tick) {
- if (self.isFunction(self.yAxisTickFormatter)) {
- return self.yAxisTickFormatter(tick);
- } else {
- return tick;
- }
- });
-
- var svg = d3.select(this.element)
- .append("svg")
- .attr("class", "simple-bar-chart")
- .attr("width", width + margin.left + margin.right + axisLabelPadding)
- .attr("height", height + margin.top + margin.bottom + axisLabelPadding)
- .append("g")
- .attr("transform", "translate("+(margin.left+axisLabelPadding)+","+(margin.top)+")");
-
- xScale.domain(data.map(function(d) { return d.label; }));
- yScale.domain([0, d3.max(data, function(d) { return d.value; })]);
-
- var gx = svg.append("g")
- .attr("class", "x axis")
- .attr("transform", "translate(0," + height + ")")
- .call(xAxis);
-
- gx.selectAll("text")
- .style("text-anchor", "end")
- .attr("dx", "-.8em")
- .attr("dy", "-.3em")
- .attr("transform", "rotate(-60)");
-
- gx.append("text")
- .attr("transform", "translate("+(width/2)+","+(margin.bottom)+")")
- .style("text-anchor", "middle")
- .text(xAxisText);
-
- var gy = svg.append("g")
- .attr("class", "y axis")
- .call(yAxis);
-
- gy.append("text")
- .attr("transform", "translate("+(-margin.left)+","+(height/2)+")rotate(-90)")
- .style("text-anchor", "middle")
- .text(yAxisText);
-
- var barWidth = this.barWidth;
- var minBarWidth = Math.min(barWidth, xScale.rangeBand());
- var bars = svg.selectAll("bar")
- .data(data)
- .enter().append("rect")
- .attr("x", function(d) {
- var padding = 0;
- var rangeBand = xScale.rangeBand();
- if ((rangeBand - barWidth) > 0) {
- padding = (rangeBand - barWidth) / 2;
- }
- return xScale(d.label) + padding;
- })
- .attr("width", minBarWidth)
- .attr("y", function() {
- return yScale(0);
- })
- .attr("height", function() {
- return height - yScale(0);
- })
- .on('click', function(d) {
- if (self.enableTooltip && self.hideTootlipOnBarChartClick) {
- self.hideTootlip();
- }
- if (self.isFunction(self.onBarChartClickCallback)) {
- self.onBarChartClickCallback(d);
- }
- });
-
- bars.transition()
- .duration(1000)
- .delay(100)
- .attr("y", function(d) {
- return yScale(d.value);
- })
- .attr("height", function(d) {
- return height - yScale(d.value);
- });
-
- if (this.enableTooltip) {
- this.bindTooltip(bars);
- }
- },
-
- bindTooltip(bars) {
- var self = this;
- var tooltip = this.tooltip;
- if (tooltip) {
- bars.on("mouseenter", function(d) {
- tooltip.html(d.tooltip);
- self.showTooltip();
- }).on("mousemove", function() {
- tooltip.style("left", (d3.event.pageX + 5) + "px")
- .style("top", (d3.event.pageY - 25) + "px");
- }).on("mouseout", function() {
- self.hideTootlip();
- });
- }
- },
-
- initTooltip() {
- this.tooltip = d3.select("body")
- .append("div")
- .attr("class", "tooltip simple-barchart-tooltip")
- .style("opacity", 1);
-
- this.hideTootlip();
- },
-
- hideTootlip() {
- if (this.tooltip) {
- this.tooltip.style("display", "none");
- }
- },
-
- showTooltip() {
- if (this.tooltip) {
- this.tooltip.style("display", "block");
- }
- },
-
- isFunction(func) {
- return Ember.typeOf(func) === "function";
- },
-
- didInsertElement() {
- this.initChart();
- if (this.enableTooltip) {
- this.initTooltip();
- }
- this.drawChart();
- },
-
- willDestroyElement() {
- if (this.tooltip) {
- this.tooltip.remove();
- }
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/simple-table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/simple-table.js
deleted file mode 100644
index c5dadc3d925cc..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/simple-table.js
+++ /dev/null
@@ -1,81 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- didInsertElement: function() {
- var paging = this.get("paging") ? true : this.get("paging");
- var ordering = this.get("ordering") ? true : this.get("ordering");
- var info = this.get("info") ? true : this.get("info");
- var bFilter = this.get("bFilter") ? true : this.get("bFilter");
- var defaultSearch = this.get("defaultSearch") ? this.get("defaultSearch") : "";
-
- // Defines sorter for the columns if not default.
- // Can also specify a custom sorter.
- var i;
- var colDefs = [];
- if (this.get("colTypes")) {
- var typesArr = this.get("colTypes").split(' ');
- var targetsArr = this.get("colTargets").split(' ');
- for (i = 0; i < typesArr.length; i++) {
- console.log(typesArr[i] + " " + targetsArr[i]);
- colDefs.push({
- type: typesArr[i],
- targets: parseInt(targetsArr[i])
- });
- }
- }
- // Defines initial column and sort order.
- var orderArr = [];
- if (this.get("colsOrder")) {
- var cols = this.get("colsOrder").split(' ');
- for (i = 0; i < cols.length; i++) {
- var col = cols[i].split(',');
- if (col.length !== 2) {
- continue;
- }
- var order = col[1].trim();
- if (order !== 'asc' && order !== 'desc') {
- continue;
- }
- var colOrder = [];
- colOrder.push(parseInt(col[0]));
- colOrder.push(order);
- orderArr.push(colOrder);
- }
- }
- if (orderArr.length === 0) {
- var defaultOrder = [0, 'asc'];
- orderArr.push(defaultOrder);
- }
- console.log(orderArr[0]);
- Ember.$('#' + this.get('table-id')).DataTable({
- "paging": paging,
- "ordering": ordering,
- "info": info,
- "bFilter": bFilter,
- "order": orderArr,
- "columnDefs": colDefs,
- "oSearch": {
- "sSearch": defaultSearch
- }
- });
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/stacked-barchart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/stacked-barchart.js
deleted file mode 100644
index e57d747e4d2ac..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/stacked-barchart.js
+++ /dev/null
@@ -1,201 +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.
- */
-
-import BaseChartComponent from 'yarn-ui/components/base-chart-component';
-
-export default BaseChartComponent.extend({
- MAX_BAR_HEIGHT: 120,
- MAX_BAR_WIDTH: 30,
- GAP: 5,
- filter: "",
- WIDTH_OF_SAMPLE: 200,
-
- bindTP: function(element) {
- element.on("mouseover", function() {
- this.tooltip
- .style("left", (d3.event.pageX) + "px")
- .style("top", (d3.event.pageY - 28) + "px");
- element.style("opacity", 1.0);
- }.bind(this))
- .on("mousemove", function() {
- // Handle pie chart case
- var text = element.attr("tooltiptext");
-
- this.tooltip.style("opacity", 0.9);
- this.tooltip.html(text)
- .style("left", (d3.event.pageX) + "px")
- .style("top", (d3.event.pageY - 28) + "px");
- }.bind(this))
- .on("mouseout", function() {
- this.tooltip.style("opacity", 0);
- element.style("opacity", 0.8);
- }.bind(this));
-
- element.on("click", function() {
- if (element.attr("link")) {
- this.tooltip.remove();
- document.location.href = element.attr("link");
- }
- }.bind(this));
- },
-
- printSamples: function(n, layout, g, colorTitles) {
- var yOffset = layout.margin * 3;
-
- for (var i = 0; i < n; i++) {
- var xOffset = layout.x2 - this.WIDTH_OF_SAMPLE - layout.margin;
- g.append("rect").
- attr("fill", this.colors[i]).
- attr("x", xOffset).
- attr("y", yOffset).
- attr("width", 20).
- attr("height", 20);
-
- g.append("text").
- attr("x", xOffset + 30).
- attr("y", yOffset + 10).
- text(colorTitles[i]);
-
- yOffset = yOffset + 30;
- }
- },
-
- // data:
- // [[{value=xx, bindText=xx}, {value=yy, bindText=yy}], [ ... ]]
- // __________________________________________________ ___________
- // bar-1 bar-2
- show: function (data, title, colorTitles) {
- var width = this.MAX_BAR_WIDTH;
- var height = this.MAX_BAR_HEIGHT;
-
- this.chart.g.remove();
- this.chart.g = this.chart.svg.append("g");
- var g = this.chart.g;
- var layout = this.getLayout();
- layout.margin = 50;
-
- var nBarPerRow = Math.floor((layout.x2 - layout.x1 - 3 * layout.margin -
- this.WIDTH_OF_SAMPLE) /
- (width + this.GAP));
-
- var xOffset;
- var yOffset = layout.margin * 2;
-
- var maxValue = 0;
- var maxN = 0;
-
- var i = 0;
- var j = 0;
-
- for (i = 0; i < data.length; i++) {
- var total = 0;
- for (j = 0; j < data[i].length; j++) {
- total += data[i][j].value;
- }
-
- if (total > maxValue) {
- maxValue = total;
- }
- if (data[i].length > maxN) {
- maxN = data[i].length;
- }
- }
-
- // print samples
- this.printSamples(maxN, layout, g, colorTitles);
-
- // print data
- data.sort(function(a, b) {
- return b[0].value - a[0].value;
- });
-
- for (i = 0; i < data.length; i++) {
- if (i % nBarPerRow === 0) {
- xOffset = layout.margin;
- yOffset += layout.margin + height;
- }
-
- var leftTopY = yOffset;
- for (j = 0; j < data[i].length; j++) {
- var dy = data[i][j].value * height / maxValue;
- if (dy > 0) {
- leftTopY = leftTopY - dy;
-
- var node = g.append("rect").
- attr("fill", this.colors[j]).
- attr("x", xOffset).
- attr("y", leftTopY).
- attr("width", width).
- attr("height", dy).
- attr("tooltiptext",
- (data[i][j].bindText) ? data[i][j].bindText : data[i][j].value).
- attr("link", data[i][j].link)
- .style("opacity", 0.8);
-
- this.bindTP(node);
- }
- }
-
- if (data[i].length === 1) {
- g.append("text")
- .text(data[i][0].value)
- .attr("y", leftTopY - 10)
- .attr("x", xOffset + width / 2)
- .attr("class", "heatmap-cell")
- .style("fill", "black");
- }
-
- xOffset += width + this.GAP;
- }
-
- layout.y2 = yOffset + layout.margin;
- this.adjustMaxHeight(layout.y2);
- this.renderTitleAndBG(g, title, layout, false);
- },
-
- draw: function() {
- this.initChart(true);
- //Mock.initMockNodesData(this);
-
- // mock data
- var arr = [];
- for (var i = 0; i < 5; i++) {
- var subArr = [];
- for (var j = 0; j < Math.random() * 4 + 1; j++) {
- subArr.push({
- value : Math.abs(Math.random())
- });
- }
- arr.push(subArr);
- }
-
- this.show(
- arr, this.get("title"));
- },
-
- didInsertElement: function () {
- this.draw();
- },
-
- actions: {
- applyFilter: function(event) {
- this.filter = event.srcElement.value;
- this.didInsertElement();
- }
- }
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/sunburst-chart.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/sunburst-chart.js
deleted file mode 100644
index 3a2302a2fc36d..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/sunburst-chart.js
+++ /dev/null
@@ -1,261 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
-
- didInsertElement: function() {
- var json = buildHierarchy(this.get("arr"));
- createVisualization(json);
- },
-
-});
-// Dimensions of sunburst.
-var width = 1000;
-var height = 750;
-var radius = Math.min(width, height) / 2;
-// Breadcrumb dimensions: width, height, spacing, width of tip/tail.
-var b = { w: 225, h: 30, s: 3, t: 10 };
-// Total size of all segments; we set this later, after loading the data.
-var totalSize = 0;
-var vis = null;
-var partition = null;
-var arc = null;
-var colors = d3.scale.category20c();
-
-function colorMap(d) {
- return colors(d.name);
-}
-
-// Given a node in a partition layout, return an array of all of its ancestor
-// nodes, highest first, but excluding the root.
-function getAncestors(node) {
- var path = [];
- var current = node;
- while (current.parent) {
- path.unshift(current);
- current = current.parent;
- }
- return path;
-}
-
-// Main function to draw and set up the visualization, once we have the data.
-function createVisualization(json) {
- partition = d3.layout.partition()
- .size([2 * Math.PI, radius * radius])
- .value(function(d) { return d.size; });
- arc = d3.svg.arc()
- .startAngle(function(d) { return d.x; })
- .endAngle(function(d) { return d.x + d.dx; })
- .innerRadius(function(d) { return Math.sqrt(d.y); })
- .outerRadius(function(d) { return Math.sqrt(d.y + d.dy); });
- // Basic setup of page elements.
- initializeBreadcrumbTrail();
- //drawLegend();
- //d3.select("#togglelegend").on("click", toggleLegend);
-
- // Bounding circle underneath the sunburst, to make it easier to detect
- // when the mouse leaves the parent g.
- vis = d3.select("#chart").append("svg:svg")
- .attr("width", width)
- .attr("height", height)
- .append("svg:g")
- .attr("id", "container")
- .attr("transform", "translate(" + width / 2 + "," + height / 2 + ")");
- vis.append("svg:circle").attr("r", radius)
- .style("opacity", 0);
-
- // For efficiency, filter nodes to keep only those large enough to see.
- var nodes = partition.nodes(json)
- .filter(function(d) {
- return (d.dx > 0.005); // 0.005 radians = 0.29 degrees
- });
-
- var path = vis.data([json]).selectAll("path")
- .data(nodes)
- .enter().append("svg:path")
- .attr("display", function(d) { return d.depth ? null : "none"; })
- .attr("d", arc)
- .attr("fill-rule", "evenodd")
- .attr("fill", colorMap)
- .style("opacity", 1)
- .on("mouseover", mouseover);
- // Add the mouseleave handler to the bounding circle.
- d3.select("#container").on("mouseleave", mouseleave);
-
- // Get total size of the tree = value of root node from partition.
- totalSize = path.node().__data__.value;
-}
-
- // Take a 2-column CSV and transform it into a hierarchical structure suitable
- // for a partition layout. The first column is a sequence of step names, from
- // root to leaf, separated by hyphens. The second column is a count of how
- // often that sequence occurred.
-function buildHierarchy(csv) {
- var root = {"name": "root", "children": []};
- for (var i = 0; i < csv.length; i++) {
- var sequence = csv[i][0];
- var size = +csv[i][1];
- if (isNaN(size)) { // e.g. if this is a header row
- continue;
- }
- var parts = sequence.split("-");
- var currentNode = root;
- for (var j = 0; j < parts.length; j++) {
- var children = currentNode["children"];
- var nodeName = parts[j];
- var childNode;
- if (j + 1 < parts.length) {
- // Not yet at the end of the sequence; move down the tree.
- var foundChild = false;
- for (var k = 0; k < children.length; k++) {
- if (children[k]["name"] === nodeName) {
- childNode = children[k];
- foundChild = true;
- break;
- }
- }
- // If we don't already have a child node for this branch, create it.
- if (!foundChild) {
- childNode = {"name": nodeName, "children": []};
- children.push(childNode);
- }
- currentNode = childNode;
- } else {
- // Reached the end of the sequence; create a leaf node.
- childNode = {"name": nodeName, "size": size, "children": [], "sequence": sequence};
- children.push(childNode);
- }
- }
- }
- return root;
-}
-
-// Fade all but the current sequence, and show it in the breadcrumb trail.
-function mouseover(d) {
- //var percentage = (100 * d.value / totalSize).toPrecision(3);
- //var percentageString = percentage + "%";
- var percentageString = d.value;
-
- d3.select("#percentage").html('
' + percentageString +
- '
(' + d.name + ')
');
- d3.select("#explanation").style("visibility", "");
-
- var sequenceArray = getAncestors(d);
- updateBreadcrumbs(sequenceArray, percentageString);
-
- // Fade all the segments.
- d3.selectAll("path").style("opacity", 0.2);
-
- // Then highlight only those that are an ancestor of the current segment.
- vis.selectAll("path")
- .filter(function(node) {
- return (sequenceArray.indexOf(node) >= 0);
- })
- .style("opacity", 1);
-}
-
-// Restore everything to full opacity when moving off the visualization.
-function mouseleave() {
- // Hide the breadcrumb trail
- d3.select("#trail").style("visibility", "hidden");
-
- // Deactivate all segments during transition.
- d3.selectAll("path").on("mouseover", null);
-
- // Transition each segment to full opacity and then reactivate it.
- d3.selectAll("path")
- .transition()
- .duration(1000)
- .style("opacity", 1)
- .each("end", function() {
- d3.select(this).on("mouseover", mouseover);
- });
-
- d3.select("#explanation")
- .style("visibility", "hidden");
-}
-
-function initializeBreadcrumbTrail() {
- // Add the svg area.
- var trail = d3.select("#sequence").append("svg:svg")
- .attr("width", width)
- .attr("height", 50)
- .attr("id", "trail");
- // Add the label at the end, for the percentage.
- trail.append("svg:text")
- .attr("id", "endlabel")
- .style("fill", "#000");
-}
-
-// Generate a string that describes the points of a breadcrumb polygon.
-function breadcrumbPoints(d, i) {
- var points = [];
- points.push("0,0");
- points.push(b.w + ",0");
- points.push(b.w + b.t + "," + (b.h / 2));
- points.push(b.w + "," + b.h);
- points.push("0," + b.h);
- if (i > 0) { // Leftmost breadcrumb; don't include 6th vertex.
- points.push(b.t + "," + (b.h / 2));
- }
- return points.join(" ");
-}
-
-// Update the breadcrumb trail to show the current sequence and percentage.
-function updateBreadcrumbs(nodeArray, percentageString) {
- // Data join; key function combines name and depth (= position in sequence).
- var g = d3.select("#trail")
- .selectAll("g")
- .data(nodeArray, function(d) { return d.name + d.depth; });
-
- // Add breadcrumb and label for entering nodes.
- var entering = g.enter().append("svg:g");
-
- entering.append("svg:polygon")
- .attr("points", breadcrumbPoints)
- .attr("fill", colorMap);
-
- entering.append("svg:text")
- .attr("x", (b.w + b.t) / 2)
- .attr("y", b.h / 2)
- .attr("dy", "0.35em")
- .attr("text-anchor", "middle")
- .text(function(d) { return d.name; });
-
- // Set position for entering and updating nodes.
- g.attr("transform", function(d, i) {
- return "translate(" + i * (b.w + b.s) + ", 0)";
- });
-
- // Remove exiting nodes.
- g.exit().remove();
-
- // Now move and update the percentage at the end.
- d3.select("#trail").select("#endlabel")
- .attr("x", (nodeArray.length + 0.5) * (b.w + b.s))
- .attr("y", b.h / 2)
- .attr("dy", "0.35em")
- .attr("text-anchor", "middle")
- .text(percentageString);
-
- // Make the breadcrumb trail visible, if it's hidden.
- d3.select("#trail")
- .style("visibility", "");
-}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
deleted file mode 100644
index 031dd41be9390..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/timeline-view.js
+++ /dev/null
@@ -1,546 +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.
- */
-
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-import ColumnDef from 'em-table/utils/column-definition';
-import TableDefinition from 'em-table/utils/table-definition';
-
-export default Ember.Component.extend({
- tableDefinition: TableDefinition.create({
- searchType: 'manual',
- }),
- graphDrawn: false,
- userInfo: null,
-
- actions: {
- changeViewType(param) {
- this.sendAction("changeViewType", param);
- if (this.get('attemptModel')) {
- this.setAttemptsGridColumnsAndRows();
- }
- }
- },
-
- canvas: {
- svg: undefined,
- h: 0,
- w: 0,
- tooltip: undefined
- },
-
- clusterMetrics: undefined,
- modelArr: [],
- containerIdArr: [],
- colors: d3.scale.category10().range(),
- _selected: undefined,
- gridColumns: [],
- gridRows: [],
- serviceName: undefined,
-
- selected: function() {
- return this._selected;
- }.property(),
-
- tableComponentName: function() {
- return "app-attempt-table";
- }.property(),
-
- setSelected: function(d) {
- var dom;
-
- if (this._selected === d) {
- return;
- }
-
- // restore color
- if (this._selected) {
- dom = d3.select("#timeline-bar-" + this._selected.get("id"));
- dom.attr("fill", this.colors[0]);
- }
-
- this._selected = d;
- this.set("selected", d);
- dom = d3.select("#timeline-bar-" + d.get("id"));
- dom.attr("fill", this.colors[1]);
- },
-
- getPerItemHeight: function() {
- var arrSize = this.modelArr.length;
-
- if (arrSize < 20) {
- return 30;
- } else if (arrSize < 100) {
- return 10;
- } else {
- return 2;
- }
- },
-
- getPerItemGap: function() {
- var arrSize = this.modelArr.length;
-
- if (arrSize < 20) {
- return 5;
- } else if (arrSize < 100) {
- return 1;
- } else {
- return 1;
- }
- },
-
- getCanvasHeight: function() {
- return (this.getPerItemHeight() + this.getPerItemGap()) * this.modelArr.length + 200;
- },
-
- draw: function(start, end) {
- // get w/h of the svg
- var bbox = d3.select("#" + this.get("parent-id"))
- .node()
- .getBoundingClientRect();
- this.canvas.w = bbox.width;
- this.canvas.h = this.getCanvasHeight();
-
- this.canvas.svg = d3.select("#" + this.get("parent-id"))
- .append("svg")
- .attr("width", this.canvas.w)
- .attr("height", this.canvas.h)
- .attr("id", this.get("my-id"));
- this.renderTimeline(start, end);
- },
-
- renderTimeline: function(start, end) {
- var border = 30;
- var singleBarHeight = this.getPerItemHeight();
- var gap = this.getPerItemGap();
- var textWidth = 200;
- /*
- start-time end-time
- |--------------------------------------|
- ==============
- ==============
- ==============
- ===============
- */
- var xScaler = d3.scale.linear()
- .domain([start, end])
- .range([0, this.canvas.w - 2 * border - textWidth]);
-
- /*
- * Render frame of timeline view
- */
- this.canvas.svg.append("line")
- .attr("x1", border + textWidth)
- .attr("y1", border - 5)
- .attr("x2", this.canvas.w - border)
- .attr("y2", border - 5)
- .attr("class", "chart");
-
- this.canvas.svg.append("line")
- .attr("x1", border + textWidth)
- .attr("y1", border - 10)
- .attr("x2", border + textWidth)
- .attr("y2", border - 5)
- .attr("class", "chart");
-
- this.canvas.svg.append("line")
- .attr("x1", this.canvas.w - border)
- .attr("y1", border - 10)
- .attr("x2", this.canvas.w - border)
- .attr("y2", border - 5)
- .attr("class", "chart");
-
- this.canvas.svg.append("text")
- .text(Converter.timeStampToDate(start))
- .attr("y", border - 15)
- .attr("x", border + textWidth)
- .attr("class", "bar-chart-text")
- .attr("text-anchor", "left");
-
- this.canvas.svg.append("text")
- .text(Converter.timeStampToDate(end))
- .attr("y", border - 15)
- .attr("x", this.canvas.w - border)
- .attr("class", "bar-chart-text")
- .attr("text-anchor", "end");
-
- // show bar
- var bar = this.canvas.svg.selectAll("bars")
- .data(this.modelArr)
- .enter()
- .append("rect")
- .attr("y", function(d, i) {
- return border + (gap + singleBarHeight) * i;
- })
- .attr("x", function(d) {
- return border + textWidth + xScaler(d.get("startTs"));
- })
- .attr("height", singleBarHeight)
- .attr("fill", function() {
- return this.colors[0];
- }.bind(this))
- .attr("width", function(d) {
- var finishedTs = xScaler(d.get("finishedTs"));
- finishedTs = finishedTs > 0 ? finishedTs : xScaler(end);
- return finishedTs - xScaler(d.get("startTs"));
- })
- .attr("id", function(d) {
- return "timeline-bar-" + d.get("id");
- });
- bar.on("click", function(d) {
- this.setSelected(d);
- }.bind(this));
-
- this.bindTooltip(bar);
-
- if (this.modelArr.length <= 20) {
- // show bar texts
- for (var i = 0; i < this.modelArr.length; i++) {
- this.canvas.svg.append("text")
- .text(this.modelArr[i].get(this.get("label")))
- .attr("y", border + (gap + singleBarHeight) * i + singleBarHeight / 2)
- .attr("x", border)
- .attr("class", "bar-chart-text");
- }
- }
- },
-
- bindTooltip: function(d) {
- d.on("mouseover", function() {
- this.tooltip
- .style("left", (d3.event.pageX) + "px")
- .style("top", (d3.event.pageY - 28) + "px");
- }.bind(this))
- .on("mousemove", function(d) {
- this.tooltip.style("opacity", 0.9);
- this.tooltip.html(d.get("tooltipLabel"))
- .style("left", (d3.event.pageX) + "px")
- .style("top", (d3.event.pageY - 28) + "px");
- }.bind(this))
- .on("mouseout", function() {
- this.tooltip.style("opacity", 0);
- }.bind(this));
- },
-
- initTooltip: function() {
- this.tooltip = d3.select("body")
- .append("div")
- .attr("class", "tooltip")
- .attr("id", "chart-tooltip")
- .style("opacity", 0);
- },
-
- didInsertElement: function() {
- // init model
- this.modelArr = [];
- this.containerIdArr = [];
-
- if (this.get("rmModel")) {
- this.get("rmModel").forEach(function(o) {
- if(!this.modelArr.contains(o)) {
- this.modelArr.push(o);
- this.containerIdArr.push(o.id);
- }
- }.bind(this));
- }
-
- if (this.get("tsModel")) {
- this.get("tsModel").forEach(function(o) {
- if(!this.containerIdArr.contains(o.id)) {
- this.modelArr.push(o);
- }
- }.bind(this));
- }
-
- if (this.modelArr.length === 0) {
- return;
- }
-
- this.modelArr.sort(function(a, b) {
- var tsA = a.get("startTs");
- var tsB = b.get("startTs");
- return tsA - tsB;
- });
-
- if (this.get('attemptModel')) {
- this.setAttemptsGridColumnsAndRows();
- } else {
- this.setContainersGridColumnsAndRows();
- }
- },
-
- didUpdate: function() {
- if (this.get("viewType") === "grid" || this.graphDrawn) {
- return;
- }
-
- this.initTooltip();
-
- var begin = 0;
- if (this.modelArr.length > 0) {
- begin = this.modelArr[0].get("startTs");
- }
- var end = 0;
- for (var i = 0; i < this.modelArr.length; i++) {
- var ts = this.modelArr[i].get("finishedTs");
- if (ts > end) {
- end = ts;
- }
- }
- if (end < begin) {
- end = Date.now();
- }
-
- this.draw(begin, end);
-
- if (this.modelArr.length > 0) {
- this.setSelected(this.modelArr[0]);
- }
-
- this.graphDrawn = true;
- },
-
- setAttemptsGridColumnsAndRows: function() {
- var self = this;
- var columns = [];
- var serviceName = this.get('serviceName');
-
- columns.push({
- id: 'id',
- headerTitle: 'Attempt ID',
- contentPath: 'id',
- cellComponentName: 'em-table-linked-cell',
- minWidth: '300px',
- getCellContent: function(row) {
- var attemptId = row.get('id');
- var query = 'viewType=' + self.get("viewType");
- if (serviceName) {
- query += '&service=' + serviceName;
- }
- return {
- displayText: attemptId,
- href: `#/yarn-app-attempt/${attemptId}?${query}`
- };
- }
- }, {
- id: 'attemptStartedTime',
- headerTitle: 'Started Time',
- contentPath: 'attemptStartedTime'
- }, {
- id: 'finishedTime',
- headerTitle: 'Finished Time',
- contentPath: 'finishedTime',
- getCellContent: function(row) {
- if (row.get('finishedTs')) {
- return row.get('finishedTime');
- }
- return 'N/A';
- }
- }, {
- id: 'elapsedTime',
- headerTitle: 'Elapsed Time',
- contentPath: 'elapsedTime'
- }, {
- id: 'appMasterContainerId',
- headerTitle: 'AM Container ID',
- contentPath: 'appMasterContainerId',
- minWidth: '350px'
- }, {
- id: 'amNodeId',
- headerTitle: 'AM Node ID',
- contentPath: 'amNodeId'
- }, {
- id: 'attemptState',
- headerTitle: 'State',
- contentPath: 'attemptState',
- getCellContent: function(row) {
- var state = row.get('attemptState');
- if (state) {
- return state;
- } else {
- return 'N/A';
- }
- }
- }, {
- id: 'exposedPorts',
- headerTitle: 'Exposed Ports',
- contentPath: 'exposedPorts',
- getCellContent: function(row) {
- var ports = row.get('exposedPorts');
- if (ports) {
- return ports;
- } else {
- return 'N/A';
- }
- }
- }, {
- id: 'nodeHttpAddress',
- headerTitle: 'NodeManager Web UI',
- contentPath: 'nodeHttpAddress',
- cellComponentName: 'em-table-html-cell',
- getCellContent: function(row) {
- var address = self.checkHttpProtocol(row.get('nodeHttpAddress'));
- var link = row.get('masterNodeURL');
- if (address) {
- return `${address}`;
- } else {
- return 'N/A';
- }
- }
- }, {
- id: 'logsLink',
- headerTitle: 'Logs',
- contentPath: 'logsLink',
- cellComponentName: 'em-table-html-cell',
- getCellContent: function(row) {
- var containerLogUrl = row.get('appAttemptContainerLogsURL');
- if (containerLogUrl) {
- return `Link`;
- } else {
- return 'N/A';
- }
- }
- });
-
- var gridCols = ColumnDef.make(columns);
- this.set('gridColumns', gridCols);
- this.set('gridRows', this.modelArr);
- },
-
- setContainersGridColumnsAndRows: function() {
- var self = this;
- var columns = [];
-
- columns.push({
- id: 'id',
- headerTitle: 'Container ID',
- contentPath: 'id',
- cellComponentName: 'em-table-html-cell',
- minWidth: '350px',
- getCellContent: function(row) {
- var termLink = self.checkHttpProtocol(row.get('nodeHttpAddress'));
- var containerId = row.get('id');
- var requestedUser = self.get('requestedUser');
- return `${containerId}`;
- }
- }, {
- id: 'startedTime',
- headerTitle: 'Started Time',
- contentPath: 'startedTime'
- }, {
- id: 'finishedTime',
- headerTitle: 'Finished Time',
- contentPath: 'finishedTime',
- getCellContent: function(row) {
- if (row.get('finishedTs')) {
- return row.get('finishedTime');
- }
- return 'N/A';
- }
- }, {
- id: 'elapsedTime',
- headerTitle: 'Elapsed Time',
- contentPath: 'elapsedTime'
- }, {
- id: 'priority',
- headerTitle: 'Priority',
- contentPath: 'priority'
- }, {
- id: 'containerExitStatus',
- headerTitle: 'Exit Status',
- contentPath: 'containerExitStatus',
- getCellContent: function(row) {
- var status = row.get('containerExitStatus');
- if (status) {
- return status;
- } else {
- return 'N/A';
- }
- }
- }, {
- id: 'containerState',
- headerTitle: 'State',
- contentPath: 'containerState',
- getCellContent: function(row) {
- var state = row.get('containerState');
- if (state) {
- return state;
- } else {
- return 'N/A';
- }
- }
- }, {
- id: 'logUrl',
- headerTitle: 'Logs',
- contentPath: 'logUrl',
- cellComponentName: 'em-table-html-cell',
- getCellContent: function(row) {
- var containerLogUrl = row.get('appAttemptContainerLogsURL');
- if (containerLogUrl) {
- return `Link`;
- } else {
- return 'N/A';
- }
- }
- }, {
- id: 'exposedPorts',
- headerTitle: 'Exposed Ports',
- contentPath: 'exposedPorts',
- getCellContent: function(row) {
- var ports = row.get('exposedPorts');
- if (ports) {
- return ports;
- } else {
- return 'N/A';
- }
- }
- }, {
- id: 'nodeHttpAddress',
- headerTitle: 'Node Manager UI',
- contentPath: 'nodeHttpAddress',
- cellComponentName: 'em-table-html-cell',
- getCellContent: function(row) {
- var address = self.checkHttpProtocol(row.get('nodeHttpAddress'));
- var link = row.get('masterNodeURL');
- if (address) {
- return `${address}`;
- } else {
- return 'N/A';
- }
- }
- });
-
- var gridCols = ColumnDef.make(columns);
- this.set('gridColumns', gridCols);
- this.set('gridRows', this.modelArr);
- },
-
- checkHttpProtocol: function(prop) {
- if (prop && prop.indexOf('://') < 0) {
- prop = 'http://' + prop;
- }
- return prop;
- },
-
- requestedUser: function() {
- if (this.get('userInfo')) {
- return this.get('userInfo.requestedUser');
- }
- return '';
- }.property('userInfo'),
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
deleted file mode 100644
index 9becfc10621d6..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
+++ /dev/null
@@ -1,412 +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.
- */
-
-import Ember from "ember";
-import {PARTITION_LABEL} from '../constants';
-
-const INBETWEEN_HEIGHT = 130;
-
-export default Ember.Component.extend({
- // Map:
- map: undefined,
-
- // Normalized data for d3
- treeData: undefined,
-
- // folded queues, folded[] == true means is folded
- foldedQueues: {},
-
- // maxDepth
- maxDepth: 0,
-
- // num of leaf queue, folded queue is treated as leaf queue
- numOfLeafQueue: 0,
-
- // mainSvg
- mainSvg: undefined,
-
- used: undefined,
- max: undefined,
-
- didUpdateAttrs: function({ oldAttrs, newAttrs }) {
- if (oldAttrs.filteredPartition.value !== newAttrs.filteredPartition.value) {
- this.reDraw();
- }
- },
- // Init data
- initData: function() {
- this.map = {};
- this.treeData = {};
- this.maxDepth = 0;
- this.numOfLeafQueue = 0;
-
- this.get("model").forEach(
- function(o) {
- this.map[o.id] = o;
- }.bind(this)
- );
-
- // var selected = this.get("selected");
- this.used = this.get("used");
- this.max = this.get("max");
- this.initQueue("root", 1, this.treeData);
- },
-
- // get Children array of given queue
- getChildrenNamesArray: function(q) {
- var namesArr = [];
-
- // Folded queue's children is empty
- if (this.foldedQueues[q.get("name")]) {
- return namesArr;
- }
-
- var names = q.get("children");
- if (names) {
- names.forEach(function(name) {
- namesArr.push(name);
- });
- }
-
- return namesArr;
- },
-
- // Init queues
- initQueue: function(queueName, depth, node) {
- if (!queueName || !this.map[queueName]) {
- // Queue is not existed
- return false;
- }
- if (depth > this.maxDepth) {
- this.maxDepth = this.maxDepth + 1;
- }
-
- var queue = this.map[queueName];
-
- if (
- this.filteredPartition &&
- !queue.get("partitions").contains(this.filteredPartition)
- ) {
- return false;
- }
-
- var names = this.getChildrenNamesArray(queue);
-
- node.name = queueName;
- node.parent = queue.get("parent");
- node.queueData = queue;
-
- if (names.length > 0) {
- node.children = [];
-
- names.forEach(
- function(name) {
- var childQueueData = {};
- node.children.push(childQueueData);
- const status = this.initQueue(name, depth + 1, childQueueData);
- if (!status) {
- node.children.pop();
- }
- }.bind(this)
- );
- } else {
- this.numOfLeafQueue = this.numOfLeafQueue + 1;
- }
-
- return true;
- },
-
- update: function(source, root, tree, diagonal) {
- var duration = 300;
- var i = 0;
-
- // Compute the new tree layout.
- var nodes = tree.nodes(root).reverse();
- var links = tree.links(nodes);
-
- // Normalize for fixed-depth.
- nodes.forEach(function(d) {
- d.y = d.depth * 200;
- });
-
- // Update the nodes…
- var node = this.mainSvg.selectAll("g.node").data(nodes, function(d) {
- return d.id || (d.id = ++i);
- });
-
- // Enter any new nodes at the parent's previous position.
- var nodeEnter = node
- .enter()
- .append("g")
- .attr("class", "node")
- .attr("transform", function() {
- return `translate(${source.y0 + 50}, ${source.x0})`;
- })
- .on(
- "click",
- function(d) {
- if (d.queueData.get("name") !== this.get("selected")) {
- document.location.href =
- "#/yarn-queues/" + d.queueData.get("name") + "!";
- }
-
- Ember.run.later(
- this,
- function() {
- var treeWidth = this.maxDepth * 200;
- var treeHeight = this.numOfLeafQueue * INBETWEEN_HEIGHT;
- var tree = d3.layout.tree().size([treeHeight, treeWidth]);
- var diagonal = d3.svg.diagonal().projection(function(d) {
- return [d.y + 50, d.x];
- });
-
- this.update(this.treeData, this.treeData, tree, diagonal);
- },
- 100
- );
- }.bind(this)
- )
- .on("dblclick", function(d) {
- document.location.href =
- "#/yarn-queue/" + d.queueData.get("name") + "/apps";
- });
-
- nodeEnter
- .append("circle")
- .attr("r", 1e-6)
- .style(
- "fill",
- function(d) {
- const usedCapacity = getUsedCapacity(d.queueData, this.filteredPartition);
-
- if (usedCapacity <= 60.0) {
- return "#60cea5";
- } else if (usedCapacity <= 100.0) {
- return "#ffbc0b";
- } else {
- return "#ef6162";
- }
- }.bind(this)
- );
-
- // append percentage
- nodeEnter
- .append("text")
- .attr("x", function() {
- return 0;
- })
- .attr("dy", ".35em")
- .attr("fill", "white")
- .attr("text-anchor", function() {
- return "middle";
- })
- .text(
- function(d) {
- const usedCapacity = getUsedCapacity(d.queueData, this.filteredPartition);
-
- if (usedCapacity >= 100.0) {
- return usedCapacity.toFixed(0) + "%";
- } else {
- return usedCapacity.toFixed(1) + "%";
- }
- }.bind(this)
- )
- .style("fill-opacity", 1e-6);
-
- // append queue name
- nodeEnter
- .append("text")
- .attr("x", "0px")
- .attr("dy", "45px")
- .attr("text-anchor", "middle")
- .text(function(d) {
- return d.name;
- })
- .style("fill-opacity", 1e-6);
-
- // Transition nodes to their new position.
- var nodeUpdate = node
- .transition()
- .duration(duration)
- .attr("transform", function(d) {
- return `translate(${d.y + 50}, ${d.x})`;
- });
-
- nodeUpdate
- .select("circle")
- .attr("r", 30)
- .attr("href", function(d) {
- return "#/yarn-queues/" + d.queueData.get("name");
- })
- .style(
- "stroke-width",
- function(d) {
- if (d.queueData.get("name") === this.get("selected")) {
- return 7;
- } else {
- return 2;
- }
- }.bind(this)
- )
- .style(
- "stroke",
- function(d) {
- if (d.queueData.get("name") === this.get("selected")) {
- return "gray";
- } else {
- return "gray";
- }
- }.bind(this)
- );
-
- nodeUpdate.selectAll("text").style("fill-opacity", 1);
-
- // Transition exiting nodes to the parent's new position.
- var nodeExit = node
- .exit()
- .transition()
- .duration(duration)
- .attr("transform", function() {
- return `translate(${source.y}, ${source.x})`;
- })
- .remove();
-
- nodeExit.select("circle").attr("r", 1e-6);
-
- nodeExit.select("text").style("fill-opacity", 1e-6);
-
- // Update the links…
- var link = this.mainSvg.selectAll("path.link").data(links, function(d) {
- return d.target.id;
- });
-
- // Enter any new links at the parent's previous position.
- link
- .enter()
- .insert("path", "g")
- .attr("class", "link")
- .attr("d", function() {
- var o = { x: source.x0, y: source.y0 + 50 };
- return diagonal({ source: o, target: o });
- });
-
- // Transition links to their new position.
- link
- .transition()
- .duration(duration)
- .attr("d", diagonal);
-
- // Transition exiting nodes to the parent's new position.
- link
- .exit()
- .transition()
- .duration(duration)
- .attr("d", function() {
- var o = { x: source.x, y: source.y };
- return diagonal({ source: o, target: o });
- })
- .remove();
-
- // Stash the old positions for transition.
- nodes.forEach(function(d) {
- d.x0 = d.x;
- d.y0 = d.y;
- });
- },
-
- reDraw: function() {
- this.initData();
-
- var margin = { top: 20, right: 120, bottom: 20, left: 120 };
- var treeWidth = this.maxDepth * 200;
- var treeHeight = this.numOfLeafQueue * INBETWEEN_HEIGHT;
- var width = treeWidth + margin.left + margin.right;
- var height = treeHeight + margin.top + margin.bottom;
-
- if (this.mainSvg) {
- this.mainSvg.selectAll("*").remove();
- } else {
- this.mainSvg = d3
- .select("#" + this.get("parentId"))
- .append("svg")
- .attr("width", width)
- .attr("height", height)
- .attr("class", "tree-selector");
- }
-
- this.mainSvg
- .append("g")
- .attr("transform", "translate(" + margin.left + "," + margin.top + ")");
-
- var tree = d3.layout.tree().size([treeHeight, treeWidth]);
-
- var diagonal = d3.svg.diagonal().projection(function(d) {
- return [d.y + 50, d.x];
- });
-
- var root = this.treeData;
- root.x0 = height / 2;
- root.y0 = 0;
-
- d3.select(window.frameElement).style("height", height);
-
- this.update(root, root, tree, diagonal);
- },
-
- didInsertElement: function() {
- this.reDraw();
- }
-});
-
-
-const getUsedCapacity = (queueData, filter=PARTITION_LABEL) => {
-
- const type = queueData.get("type");
- var result;
-
- switch (type) {
- case "capacity":
- const partitionMap = queueData.get("partitionMap");
- if (null == partitionMap || null == partitionMap[filter] || null == partitionMap[filter].absoluteUsedCapacity) {
- result = 0.0;
- } else {
- result = partitionMap[filter].absoluteUsedCapacity;
- }
- break;
-
- case "fair":
- if (null == queueData.get("fairResources") || null == queueData.get("fairResources").memory || null == queueData.get("usedResources") || null == queueData.get("usedResources").memory || 0 == queueData.get("fairResources").memory) {
- result = 0.0;
- } else {
- result = queueData.get("usedResources").memory / queueData.get("fairResources").memory * 100;
- }
- break;
-
- case "fifo":
- if (null == queueData.get("usedCapacity") || (null == queueData.get("capacity")) || (queueData.get("capacity") == 0)) {
- result = 0.0;
- } else {
- result = queueData.get("usedCapacity") / queueData.get("capacity") * 100;
- }
- break;
-
- default:
- result = 0.0;
- }
- return result;
-};
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/upload-config.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/upload-config.js
deleted file mode 100644
index 2f9dc9c1a98b8..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/upload-config.js
+++ /dev/null
@@ -1,54 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Component.extend({
- dialogId: "config_upload_modal",
- title: "Upload Configuration",
- configJson: '',
- parseErrorMsg: '',
-
- actions: {
- uploadConfig() {
- var json = this.get('configJson');
- try {
- JSON.parse(json);
- this.upateParseResults("");
- } catch (ex) {
- this.upateParseResults("Invalid JSON: " + ex.message);
- throw ex;
- }
- if (!this.get('parseErrorMsg')) {
- this.sendAction("uploadConfig", json);
- }
- }
- },
-
- didInsertElement() {
- this.$('#' + this.get('dialogId')).on('shown.bs.modal', function() {
- this.upateParseResults("");
- }.bind(this));
- },
-
- isValidConfigJson: Ember.computed.notEmpty('configJson'),
-
- upateParseResults(message) {
- this.set('parseErrorMsg', message);
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/yarn-queue-partition-capacity-labels.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/yarn-queue-partition-capacity-labels.js
deleted file mode 100644
index e7f9c03f35649..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/yarn-queue-partition-capacity-labels.js
+++ /dev/null
@@ -1,48 +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.
- */
-
-import Ember from "ember";
-import { PARTITION_LABEL } from "../constants";
-
-export default Ember.Component.extend({
- didUpdateAttrs: function({ oldAttrs, newAttrs }) {
- this._super(...arguments);
- this.set("data", this.initData());
- },
-
- init() {
- this._super(...arguments);
- this.set("data", this.initData());
- },
-
- initData() {
- const queue = this.get("queue");
- const partitionMap = this.get("partitionMap");
- const filteredParition = this.get("filteredPartition") || PARTITION_LABEL;
- const userLimit = queue.get("userLimit");
- const userLimitFactor = queue.get("userLimitFactor");
- const isLeafQueue = queue.get("isLeafQueue");
-
- return {
- ...partitionMap[filteredParition],
- userLimit,
- userLimitFactor,
- isLeafQueue
- };
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/config.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/config.js
deleted file mode 100644
index 157d48b858fb8..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/config.js
+++ /dev/null
@@ -1,24 +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.
- */
-
-/**
- * Host and port configurations
- */
-
-export default {
-};
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js
deleted file mode 100644
index 6b37b7f4d3491..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js
+++ /dev/null
@@ -1,39 +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.
- */
-
-/**
- * Application level global constants go here.
- */
-export default {
- PARAM_SEPARATOR: '!',
-};
-
-const BASE_UNIT = 1024
-
-export const Type = 'type';
-export const Memory = 'memory';
-export const Resource = 'resource';
-export const Unit = 'unit';
-export const Entities = {
- Type: 'type',
- Memory:'memory',
- Resource: 'resource',
- Unit: 'unit'
-}
-
-export const PARTITION_LABEL = 'Default partition';
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/app-table-columns.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/app-table-columns.js
deleted file mode 100644
index db0074db1d9f6..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/app-table-columns.js
+++ /dev/null
@@ -1,204 +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.
- */
-
-
-import Ember from 'ember';
-import ColumnDef from 'em-table/utils/column-definition';
-import TableDef from 'em-table/utils/table-definition';
-import Converter from 'yarn-ui/utils/converter';
-
-export default Ember.Controller.extend({
- tableDefinition: TableDef.create({
- sortColumnId: 'stTime',
- sortOrder: 'desc'
- }),
-
- columns: function() {
- var colums = [];
- colums.push({
- id: 'appId',
- headerTitle: 'Application ID',
- contentPath: 'id',
- cellComponentName: 'em-table-linked-cell',
- minWidth: "280px",
- facetType: null,
- getCellContent: function(row) {
- return {
- displayText: row.id,
- href: `#/yarn-app/${row.id}/attempts`
- };
- }
- }, {
- id: 'appType',
- headerTitle: 'Application Type',
- contentPath: 'applicationType',
- facetType: null,
- }, {
- id: 'appTag',
- headerTitle: 'Application Tag',
- contentPath: 'applicationTags',
- facetType: null,
- }, {
- id: 'appName',
- headerTitle: 'Application Name',
- cellComponentName: 'em-table-tooltip-text',
- contentPath: 'appName',
- facetType: null,
- }, {
- id: 'appUsr',
- headerTitle: 'User',
- contentPath: 'user',
- minWidth: "50px"
- }, {
- id: 'state',
- headerTitle: 'State',
- contentPath: 'state',
- cellComponentName: 'em-table-simple-status-cell',
- minWidth: "50px"
- }, {
- id: 'queue',
- headerTitle: 'Queue',
- cellComponentName: 'em-table-tooltip-text',
- contentPath: 'queue',
- }, {
- id: 'progress',
- headerTitle: 'Progress',
- contentPath: 'progress',
- cellComponentName: 'em-table-progress-cell',
- facetType: null,
- cellDefinition: {
- valueMax: 100
- }
- }, {
- id: 'stTime',
- headerTitle: 'Start Time',
- contentPath: 'startTime',
- facetType: null,
- getCellContent: function(row) {
- return row.get('formattedStartTime');
- }
- }, {
- id: 'elTime',
- headerTitle: 'Elapsed Time',
- contentPath: 'elapsedTime',
- facetType: null,
- cellDefinition: {
- type: "duration"
- }
- }, {
- id: 'finishTime',
- headerTitle: 'Finished Time',
- contentPath: 'validatedFinishedTs',
- facetType: null,
- observePath: true,
- getCellContent: function(row) {
- return row.get('formattedFinishedTime');
- }
- }, {
- id: 'priority',
- headerTitle: 'Priority',
- contentPath: 'priority',
- }, {
- id: 'cluster',
- headerTitle: '%Cluster',
- contentPath: 'clusterUsagePercentage',
- observePath: true
- });
- return ColumnDef.make(colums);
- }.property(),
-
- serviceColumns: function() {
- var colums = [];
- colums.push({
- id: 'appName',
- headerTitle: 'Service Name',
- contentPath: 'appName',
- minWidth: "200px",
- facetType: null,
- cellComponentName: 'em-table-linked-cell',
- getCellContent: function(row) {
- return {
- displayText: row.get('appName'),
- href: `#/yarn-app/${row.id}/components?service=${row.get('appName')}`
- };
- }
- }, {
- id: 'appId',
- headerTitle: 'Application Tag',
- contentPath: 'id',
- facetType: null,
- cellComponentName: 'em-table-tooltip-text',
- minWidth: "250px"
- }, {
- id: 'appTag',
- headerTitle: 'Application ID',
- contentPath: 'applicationTags',
- facetType: null,
- }, {
- id: 'state',
- headerTitle: 'State',
- contentPath: 'state',
- cellComponentName: 'em-table-simple-status-cell',
- minWidth: "50px"
- }, {
- id: 'cluster',
- headerTitle: '%Cluster',
- contentPath: 'clusterUsagePercentage',
- facetType: null,
- observePath: true
- }, {
- id: 'elTime',
- headerTitle: 'Elapsed Time',
- contentPath: 'elapsedTime',
- facetType: null,
- cellDefinition: {
- type: "duration"
- },
- minWidth: "200px"
- }, {
- id: 'appUsr',
- headerTitle: 'User',
- contentPath: 'user',
- facetType: null,
- minWidth: "50px"
- }, {
- id: 'queue',
- headerTitle: 'Queue',
- contentPath: 'queue',
- cellComponentName: 'em-table-tooltip-text',
- }, {
- id: 'stTime',
- headerTitle: 'Started Time',
- contentPath: 'startTime',
- facetType: null,
- getCellContent: function(row) {
- return row.get('formattedStartTime');
- }
- }, {
- id: 'finishTime',
- headerTitle: 'Finished Time',
- contentPath: 'validatedFinishedTs',
- facetType: null,
- observePath: true,
- getCellContent: function(row) {
- return row.get('formattedFinishedTime');
- }
- });
- return ColumnDef.make(colums);
- }.property(),
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/application.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/application.js
deleted file mode 100644
index 34702aca2df24..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/application.js
+++ /dev/null
@@ -1,83 +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.
- */
-
-import Ember from 'ember';
-
-/**
- * Base controller for application.
- */
-export default Ember.Controller.extend({
- /**
- * Output main top UI menu which is common across all pages.
- * Menu item will be made active based on current path.
- */
- outputMainMenu: function(){
- var path = this.get('currentPath');
- var html = '
';
- return Ember.String.htmlSafe(html);
- }.property('currentPath'),
-
- isQueuesTabActive: function() {
- var path = this.get('currentPath');
- if (path === 'yarn-queues') {
- return true;
- }
- return false;
- }.property('currentPath'),
-
- clusterInfo: function() {
- if (this.model && this.model.clusterInfo) {
- return this.model.clusterInfo.get('firstObject');
- }
- return null;
- }.property('model.clusterInfo'),
-
- userInfo: function() {
- if (this.model && this.model.userInfo) {
- return this.model.userInfo.get('firstObject');
- }
- return null;
- }.property('model.userInfo'),
-
- isTimelineUnHealthy: function() {
- if (this.model && this.model.timelineHealth) {
- return this.model.timelineHealth.get('isTimelineUnHealthy');
- }
- return true;
- }.property('model.timelineHealth')
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/cluster-overview.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/cluster-overview.js
deleted file mode 100644
index 22e6267558938..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/cluster-overview.js
+++ /dev/null
@@ -1,32 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- loading: true,
-
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Cluster Overview",
- routeName: 'cluster-overview',
- }]
-
-});
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempt.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempt.js
deleted file mode 100644
index 08f099b0f35bd..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app-attempt.js
+++ /dev/null
@@ -1,71 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- queryParams: ["service", "viewType"],
- service: undefined,
- viewType: "graph",
-
- actions: {
- changeViewType(param) {
- this.set("viewType", param);
- }
- },
-
- breadcrumbs: Ember.computed("model.attempt.appId", "model.attempt.id", function () {
- var appId = this.get("model.attempt.appId");
- var attemptId = this.get("model.attempt.id");
- var serviceName = this.get('service');
- var breadcrumbs = [{
- text: "Home",
- routeName: 'application'
- },{
- text: "Applications",
- routeName: 'yarn-apps.apps'
- }, {
- text: `App [${appId}]`,
- href: `#/yarn-app/${appId}/attempts`
- }, {
- text: "Attempts",
- href: `#/yarn-app/${appId}/attempts`
- }, {
- text: `Attempt [${attemptId}]`
- }];
- if (serviceName) {
- breadcrumbs = [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Services",
- routeName: 'yarn-services'
- }, {
- text: `${serviceName} [${appId}]`,
- href: `#/yarn-app/${appId}/components?service=${serviceName}`
- }, {
- text: "Attempts",
- href: `#/yarn-app/${appId}/attempts?service=${serviceName}`
- }, {
- text: `Attempt [${attemptId}]`
- }];
- }
- return breadcrumbs;
- })
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
deleted file mode 100644
index 8b48347e68db3..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app.js
+++ /dev/null
@@ -1,188 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- appId: '',
- serviceName: undefined,
-
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Applications",
- routeName: 'yarn-apps.apps'
- }, {
- text: 'App'
- }],
-
- actions: {
- showStopServiceConfirm() {
- this.set('actionResponse', null);
- Ember.$("#stopServiceConfirmDialog").modal('show');
- },
-
- stopService() {
- var self = this;
- Ember.$("#stopServiceConfirmDialog").modal('hide');
- var adapter = this.store.adapterFor('yarn-servicedef');
- self.set('isLoading', true);
- adapter.stopService(this.model.serviceName, this.get('model.app.user')).then(function () {
- self.set('actionResponse', { msg: 'Service stopped successfully. Auto refreshing in 5 seconds.', type: 'success' });
- Ember.run.later(self, function () {
- this.set('actionResponse', null);
- this.send("refresh");
- }, 5000);
- }, function (errr) {
- let messg = 'Error: Stop service failed!';
- if (errr.errors && errr.errors[0] && errr.errors[0].diagnostics) {
- messg = 'Error: ' + errr.errors[0].diagnostics;
- }
- self.set('actionResponse', { msg: messg, type: 'error' });
- }).finally(function () {
- self.set('isLoading', false);
- });
- },
-
- showDeleteServiceConfirm() {
- this.set('actionResponse', null);
- Ember.$("#deleteServiceConfirmDialog").modal('show');
- },
-
- deleteService() {
- var self = this;
- Ember.$("#deleteServiceConfirmDialog").modal('hide');
- var adapter = this.store.adapterFor('yarn-servicedef');
- self.set('isLoading', true);
- adapter.deleteService(this.model.serviceName, this.get('model.app.user')).then(function () {
- self.set('actionResponse', { msg: 'Service deleted successfully. Redirecting to services in 5 seconds.', type: 'success' });
- Ember.run.later(self, function () {
- this.set('actionResponse', null);
- this.transitionToRoute("yarn-services");
- }, 5000);
- }, function (errr) {
- let messg = 'Error: Delete service failed!';
- if (errr.errors && errr.errors[0] && errr.errors[0].diagnostics) {
- messg = 'Error: ' + errr.errors[0].diagnostics;
- }
- self.set('actionResponse', { msg: messg, type: 'error' });
- }).finally(function () {
- self.set('isLoading', false);
- });
- },
-
- showKillApplicationConfirm() {
- this.set('actionResponse', null);
- Ember.$("#killApplicationConfirmDialog").modal('show');
- },
-
- killApplication() {
- var self = this;
- Ember.$("#killApplicationConfirmDialog").modal('hide');
- const adapter = this.store.adapterFor('yarn-app');
- self.set('isLoading', true);
- adapter.sendKillApplication(this.model.app.id).then(function () {
- self.set('actionResponse', {
- msg: 'Application killed successfully. Auto refreshing in 5 seconds.',
- type: 'success'
- });
- Ember.run.later(self, function () {
- this.set('actionResponse', null);
- this.send("refresh");
- }, 5000);
- }, function (err) {
- let message = err.diagnostics || 'Error: Kill application failed!';
- self.set('actionResponse', { msg: message, type: 'error' });
- }).finally(function () {
- self.set('isLoading', false);
- });
- },
-
- resetActionResponse() {
- this.set('actionResponse', null);
- }
- },
-
- isRunningService: Ember.computed('model.serviceName', 'model.app.state', function () {
- return this.model.serviceName && this.model.app.get('state') === 'RUNNING';
- }),
-
-
- updateBreadcrumbs(appId, serviceName, tailCrumbs) {
- var breadcrumbs = [{
- text: "Home",
- routeName: 'application'
- }];
- if (appId && serviceName) {
- breadcrumbs.push({
- text: "Services",
- routeName: 'yarn-services'
- }, {
- text: `${serviceName} [${appId}]`,
- href: `#/yarn-app/${appId}/components?service=${serviceName}`
- });
- } else {
- breadcrumbs.push({
- text: "Applications",
- routeName: 'yarn-apps.apps'
- }, {
- text: `App [${appId}]`,
- href: `#/yarn-app/${appId}/attempts`
- });
- }
- if (tailCrumbs) {
- breadcrumbs.pushObjects(tailCrumbs);
- }
- this.set('breadcrumbs', breadcrumbs);
- },
-
- amHostHttpAddressFormatted: Ember.computed('model.app.amHostHttpAddress', function () {
- var amHostAddress = this.get('model.app.amHostHttpAddress');
- if (amHostAddress && amHostAddress.indexOf('://') < 0) {
- amHostAddress = 'http://' + amHostAddress;
- }
- return amHostAddress;
- }),
-
- isAppKillable: Ember.computed("model.app.state", function () {
- if (this.get("model.app.applicationType") === 'yarn-service') {
- return false;
- }
- const killableStates = ['NEW', 'NEW_SAVING', 'SUBMITTED', 'ACCEPTED', 'RUNNING'];
- return killableStates.indexOf(this.get("model.app.state")) > -1;
- }),
-
- isServiceDeployedOrRunning: Ember.computed('model.serviceInfo', function() {
- const serviceInfo = this.get('model.serviceInfo');
- const stoppedStates = ['STOPPED', 'SUCCEEDED', 'FAILED'];
- if (serviceInfo) {
- return stoppedStates.indexOf(serviceInfo.get('state')) === -1;
- }
- return false;
- }),
-
- isServiceStoppped: Ember.computed('model.serviceInfo', function() {
- const serviceInfo = this.get('model.serviceInfo');
- const stoppedStates = ['STOPPED', 'SUCCEEDED'];
- if (serviceInfo) {
- return stoppedStates.indexOf(serviceInfo.get('state')) > -1;
- }
- return false;
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/attempts.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/attempts.js
deleted file mode 100644
index 32638e5da2e97..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/attempts.js
+++ /dev/null
@@ -1,31 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- queryParams: ["service", "viewType"],
- service: undefined,
- viewType: "graph",
-
- actions: {
- changeViewType(param) {
- this.set("viewType", param);
- }
- },
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/charts.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/charts.js
deleted file mode 100644
index 1078b145d16b2..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/charts.js
+++ /dev/null
@@ -1,28 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- queryParams: ["service"],
- service: undefined,
-
- isRunningApp: Ember.computed('model.app.state', function() {
- return this.get('model.app.state') === "RUNNING";
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/components.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/components.js
deleted file mode 100644
index 5a6c616caf05a..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/components.js
+++ /dev/null
@@ -1,64 +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.
- */
-
-import Ember from 'ember';
-import ColumnDef from 'em-table/utils/column-definition';
-import TableDefinition from 'em-table/utils/table-definition';
-
-export default Ember.Controller.extend({
- queryParams: ["service"],
- service: undefined,
-
- tableDefinition: TableDefinition.create({
- searchType: 'manual',
- }),
-
- tableColumns: Ember.computed('model.appId', 'model.serviceName', function() {
- var cols = [];
- var service = this.get('model.serviceName');
- var appId = this.get('model.appId');
-
- cols.push({
- id: 'name',
- headerTitle: 'Component',
- contentPath: 'name',
- cellComponentName: 'em-table-linked-cell',
- getCellContent: function(row) {
- return {
- displayText: row.get('name'),
- href: `#/yarn-component-instances/${row.get('name')}/info?service=${service}&appid=${appId}`
- };
- }
- }, {
- id: 'vcores',
- headerTitle: 'VCores',
- contentPath: 'vcores'
- }, {
- id: 'memory',
- headerTitle: 'Memory (MB)',
- contentPath: 'memory'
- }, {
- id: 'instances',
- headerTitle: 'Number of Instances',
- contentPath: 'instances',
- observePath: true
- });
-
- return ColumnDef.make(cols);
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/configs.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/configs.js
deleted file mode 100644
index a6cba9eea6238..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/configs.js
+++ /dev/null
@@ -1,24 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- queryParams: ["service"],
- service: undefined
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/info.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/info.js
deleted file mode 100644
index 68954ce173d04..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/info.js
+++ /dev/null
@@ -1,109 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- queryParams: ["service"],
- service: undefined,
- isLoading: false,
- actionResponse: null,
-
- actions: {
- showStopServiceConfirm() {
- this.set('actionResponse', null);
- Ember.$("#stopServiceConfirmDialog").modal('show');
- },
-
- stopService() {
- var self = this;
- Ember.$("#stopServiceConfirmDialog").modal('hide');
- var adapter = this.store.adapterFor('yarn-servicedef');
- self.set('isLoading', true);
- adapter.stopService(this.get('service'), this.get('model.app.user')).then(function () {
- self.set('actionResponse', { msg: 'Service stopped successfully. Auto refreshing in 5 seconds.', type: 'success' });
- Ember.run.later(self, function () {
- this.set('actionResponse', null);
- this.send("refresh");
- }, 5000);
- }, function (errr) {
- let messg = errr.diagnostics || 'Error: Stop service failed!';
- self.set('actionResponse', { msg: messg, type: 'error' });
- }).finally(function () {
- self.set('isLoading', false);
- });
- },
-
- showDeleteServiceConfirm() {
- this.set('actionResponse', null);
- Ember.$("#deleteServiceConfirmDialog").modal('show');
- },
-
- deleteService() {
- var self = this;
- Ember.$("#deleteServiceConfirmDialog").modal('hide');
- var adapter = this.store.adapterFor('yarn-servicedef');
- self.set('isLoading', true);
- adapter.deleteService(this.get('service'), this.get('model.app.user')).then(function () {
- self.set('actionResponse', { msg: 'Service deleted successfully. Redirecting to services in 5 seconds.', type: 'success' });
- Ember.run.later(self, function () {
- this.set('actionResponse', null);
- this.transitionToRoute("yarn-services");
- }, 5000);
- }, function (errr) {
- let messg = errr.diagnostics || 'Error: Delete service failed!';
- self.set('actionResponse', { msg: messg, type: 'error' });
- }).finally(function () {
- self.set('isLoading', false);
- });
- },
-
- resetActionResponse() {
- this.set('actionResponse', null);
- }
- },
-
- isRunningService: Ember.computed('model.serviceName', 'model.app.state', function () {
- return this.get('service') !== undefined && this.get('model.app.state') === 'RUNNING';
- }),
-
- amHostHttpAddressFormatted: Ember.computed('model.app.amHostHttpAddress', function () {
- var amHostAddress = this.get('model.app.amHostHttpAddress');
- if (amHostAddress && amHostAddress.indexOf('://') < 0) {
- amHostAddress = 'http://' + amHostAddress;
- }
- return amHostAddress;
- }),
-
- totalOutstandingResourceRequests: Ember.computed('model.app.resourceRequests', function() {
- const resourceRequests = this.get('model.app.resourceRequests');
- if (resourceRequests) {
- const totatResourceRequests = { memory: 0, vCores: 0 };
- [].forEach.call(resourceRequests, resource => {
- if (resource.resourceName === '*') {
- const totalMemory = resource.capability.resourceInformations.resourceInformation[0].value * resource.numContainers;
- const totalVCores = resource.capability.resourceInformations.resourceInformation[1].value * resource.numContainers;
- totatResourceRequests.memory += totalMemory;
- totatResourceRequests.vCores += totalVCores;
- }
- });
- return totatResourceRequests;
- }
- return null;
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js
deleted file mode 100644
index 4a5a444ee6267..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-app/logs.js
+++ /dev/null
@@ -1,334 +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.
- */
-
-import Ember from 'ember';
-import Constants from 'yarn-ui/constants';
-
-export default Ember.Controller.extend({
- queryParams: ["service", "attempt", "containerid"],
- service: undefined,
- attempt: undefined,
- containerid: undefined,
-
- selectedAttemptId: "",
- attemptContainerList: null,
- selectedContainerId: "",
- selectedLogFileName: "",
- containerLogFiles: null,
- selectedLogFileContent: "",
-
- _isLoadingTopPanel: false,
- _isLoadingBottomPanel: false,
-
- initializeSelect: function(selector = ".js-fetch-attempt-containers") {
- Ember.run.schedule("afterRender", this, function() {
- $(selector).select2({ width: "350px", multiple: false });
- });
- },
-
- actions: {
- showContainersForAttemptId(attemptId, containerId = "") {
- this.set("selectedAttemptId", "");
- if (attemptId) {
- this.set("_isLoadingTopPanel", true);
- this.set("selectedAttemptId", attemptId);
- this.fetchContainersForAttemptId(attemptId)
- .then(hash => {
- let containers = null;
- let containerIdArr = {};
-
- // Getting running containers from the RM first
- if (
- hash.rmContainers.get("length") > 0 &&
- hash.rmContainers.get("content")
- ) {
- hash.rmContainers.get("content").forEach(function(o) {
- containerIdArr[o.id] = true;
- }.bind(this));
- containers = (containers || []).concat(
- hash.rmContainers.get("content")
- );
- }
-
- let historyProvider = this.fallbackToJHS ? hash.jhsContainers : hash.tsContainers;
- let fieldName = this.fallbackToJHS ? "containerId" : "id";
-
- // Getting aggregated containers from the selected history provider
- if (
- historyProvider.get("length") > 0 &&
- historyProvider.get("content")
- ) {
- let historyContainers = [];
- historyProvider.get("content").forEach(function(o) {
- if(!containerIdArr[o[fieldName]]) {
- historyContainers.push(o);
- }
- }.bind(this));
- containers = (containers || []).concat(
- historyContainers);
- }
-
- this.set("attemptContainerList", containers);
- this.initializeSelect(".js-fetch-logs-containers");
- if (containerId) {
- this.send("showLogFilesForContainerId", containerId);
- }
- })
- .finally(() => {
- this.set("_isLoadingTopPanel", false);
- });
- } else {
- this.set("attemptContainerList", null);
- this.set("selectedContainerId", "");
- this.set("containerLogFiles", null);
- this.set("selectedLogFileName", "");
- this.set("selectedLogFileContent", "");
- }
- },
-
- showLogFilesForContainerId(containerId) {
- this.set("selectedContainerId", "");
- this.set("containerLogFiles", null);
- this.set("selectedLogFileName", "");
- this.set("selectedLogFileContent", "");
-
- if (containerId) {
- this.set("_isLoadingBottomPanel", true);
- this.set("selectedContainerId", containerId);
- this.fetchLogFilesForContainerId(containerId)
- .then(hash => {
- if (hash.logs.get("length") > 0) {
- this.set("containerLogFiles", hash.logs);
- } else {
- this.set("containerLogFiles", null);
- }
- this.initializeSelect(".js-fetch-log-for-container");
- })
- .finally(() => {
- this.set("_isLoadingBottomPanel", false);
- });
- }
- },
-
- showContentForLogFile(logFile) {
- this.set("selectedLogFileName", "");
- Ember.$("#logContentTextArea1234554321").val("");
- this.set("showFullLog", false);
- if (logFile) {
- this.set("_isLoadingBottomPanel", true);
- this.set("selectedLogFileName", logFile);
- var id = this.get("selectedContainerId") + Constants.PARAM_SEPARATOR + logFile;
- this.fetchContentForLogFile(id)
- .then(
- hash => {
- this.set("selectedLogFileContent", hash.logs.get('logs').trim());
- },
- () => {
- this.set("selectedLogFileContent", "");
- }
- )
- .then(() => {
- this.set("_isLoadingBottomPanel", false);
- });
- } else {
- this.set("selectedLogFileContent", "");
- }
- },
-
- findNextTextInLogContent() {
- let searchInputElem = document.getElementById("logSeachInput98765");
- this.send("searchTextInLogContent", searchInputElem.value);
- },
-
- searchTextInLogContent(searchText) {
- Ember.$("body").scrollTop(278);
- let textAreaElem = document.getElementById(
- "logContentTextArea1234554321"
- );
- let logContent = textAreaElem.innerText;
- let startIndex = this.searchTextStartIndex || 0;
- if (startIndex === -1) {
- startIndex = this.searchTextStartIndex = 0;
- }
- if (this.prevSearchText !== searchText) {
- startIndex = this.searchTextStartIndex = 0;
- }
- if (searchText && searchText.trim()) {
- searchText = searchText.trim();
- this.prevSearchText = searchText;
- if (startIndex === 0) {
- startIndex = logContent.indexOf(searchText, 0);
- }
- let endIndex = startIndex + searchText.length;
- if (document.createRange && window.getSelection) {
- let range = document.createRange();
- range.selectNodeContents(textAreaElem);
- range.setStart(textAreaElem.childNodes.item(0), startIndex);
- range.setEnd(textAreaElem.childNodes.item(0), endIndex);
- let selection = window.getSelection();
- selection.removeAllRanges();
- selection.addRange(range);
- }
- this.searchTextStartIndex = logContent.indexOf(
- searchText,
- endIndex + 1
- );
- } else {
- this.searchTextStartIndex = 0;
- }
- },
-
- showFullLogFileContent() {
- this.set("showFullLog", true);
- this.notifyPropertyChange("selectedLogFileContent");
- }
- },
-
- attemptList: Ember.computed("model.attempts", function() {
- let attempts = this.get("model.attempts");
- let list = null;
- if (attempts && attempts.get("length") && attempts.get("content")) {
- list = [].concat(attempts.get("content"));
- }
- return list;
- }),
-
- fetchContainersForAttemptId(attemptId) {
- let request = {};
-
- request["rmContainers"] = this.store
- .query("yarn-container", {
- app_attempt_id: attemptId
- })
- .catch(function(error) {
- return Ember.A();
- });
-
- let historyProvider = this.fallbackToJHS ? "jhsContainers" : "tsContainers";
- let historyQuery = this.fallbackToJHS ? "yarn-jhs-container" : "yarn-timeline-container";
-
- request[historyProvider] = this.store
- .query(historyQuery, {
- app_attempt_id: attemptId
- })
- .catch(function(error) {
- return Ember.A();
- });
-
- return Ember.RSVP.hash(request);
- },
-
- fetchLogFilesForContainerId(containerId) {
- let queryName = this.fallbackToJHS ? "yarn-jhs-log" : "yarn-log";
- let redirectQuery = queryName === "yarn-jhs-log" ? "yarn-jhs-redirect-log" : "yarn-redirect-log";
-
- return Ember.RSVP.hash({
- logs: this.resolveRedirectableQuery(
- this.store.query(queryName, { containerId }),
- m => {
- return m.map(model => model.get('redirectedUrl'))[0];
- },
- url => {
- return this.store.query(redirectQuery, url);
- })
- });
- },
-
- fetchContentForLogFile(id) {
- let queryName = this.fallbackToJHS ? 'yarn-app-jhs-log' : 'yarn-app-log';
- let redirectQuery = queryName === "yarn-app-jhs-log" ? "yarn-app-jhs-redirect-log" : "yarn-app-redirect-log";
-
- return Ember.RSVP.hash({
- logs: this.resolveRedirectableQuery(
- this.store.findRecord(queryName, id),
- m => {
- return m.get('redirectedUrl');
- },
- url => {
- return this.store.findRecord(redirectQuery, url + Constants.PARAM_SEPARATOR + id);
- })
- });
- },
-
- resolveRedirectableQuery(initial, urlResolver, redirectResolver) {
- return initial.then(m => {
- let redirectedUrl = urlResolver(m);
- if (redirectedUrl !== null && redirectedUrl !== undefined && redirectedUrl !== '') {
- let logFromRedirect = redirectResolver(redirectedUrl);
- return Promise.all([m, logFromRedirect]);
- } else {
- return Promise.all([m, null]);
- }
- })
- .then(([originalLog, logFromRedirect]) => {
- return logFromRedirect !== null ? logFromRedirect : originalLog;
- })
- .catch(function () {
- return Ember.A();
- });
- },
-
- resetAfterRefresh() {
- this.set("selectedAttemptId", "");
- this.set("attemptContainerList", null);
- this.set("selectedContainerId", "");
- this.set("selectedLogFileName", "");
- this.set("containerLogFiles", null);
- this.set("selectedLogFileContent", "");
- },
-
- showFullLog: false,
-
- showLastFewLinesOfLogContent: Ember.computed(
- "selectedLogFileContent",
- function() {
- let content = this.get("selectedLogFileContent");
- let lines = content.split("\n");
- if (this.get("showFullLog") || lines.length < 10) {
- return content;
- }
- return lines.slice(lines.length - 10).join("\n");
- }
- ),
-
- isLogAggregationNotSucceeded: Ember.computed("model.app", function() {
- const logAggregationStatus = this.get("model.app.logAggregationStatus");
- return logAggregationStatus !== "SUCCEEDED";
- }),
-
- fallbackToJHS: function() {
- // Let's fall back to JHS if ATS is not available, but JHS is.
- return this.model &&
- (!this.model.timelineHealth || this.model.timelineHealth.get('isTimelineUnHealthy')) &&
- this.model.jhsHealth && this.model.jhsHealth.get('isJHSHealthy');
- }.property('model.timelineHealth', 'model.isJHSHealthy'),
-
- areJHSandATSUnhealthy: function() {
- if (this.model && this.model.timelineHealth) {
- if (!this.model.timelineHealth.get('isTimelineUnHealthy')) {
- return false;
- }
- }
- if (this.model && this.model.jhsHealth) {
- if (this.model.jhsHealth.get('isJHSHealthy')) {
- return false;
- }
- }
- return true;
- }.property('model.timelineHealth', 'model.isJHSHealthy')
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps.js
deleted file mode 100644
index 18bf68222adb3..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps.js
+++ /dev/null
@@ -1,31 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
-
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Applications",
- routeName: 'yarn-apps.apps',
- }]
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps/apps.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps/apps.js
deleted file mode 100644
index 10965cf0e3b89..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-apps/apps.js
+++ /dev/null
@@ -1,37 +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.
- */
-
-import Ember from 'ember';
-import TableDefinition from 'em-table/utils/table-definition';
-import AppTableController from '../app-table-columns';
-
-export default AppTableController.extend({
- queryParams: ['searchText', 'sortColumnId', 'sortOrder', 'pageNum', 'rowCount'],
- tableDefinition: TableDefinition.create({
- searchType: 'manual',
- enableFaceting: true,
- rowCount: 25,
- sortColumnId: 'stTime',
- sortOrder: 'desc'
- }),
- searchText: Ember.computed.alias('tableDefinition.searchText'),
- sortColumnId: Ember.computed.alias('tableDefinition.sortColumnId'),
- sortOrder: Ember.computed.alias('tableDefinition.sortOrder'),
- pageNum: Ember.computed.alias('tableDefinition.pageNum'),
- rowCount: Ember.computed.alias('tableDefinition.rowCount')
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instance.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instance.js
deleted file mode 100644
index 61ebae1d41b75..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instance.js
+++ /dev/null
@@ -1,58 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- componentName: '',
- instanceName: '',
- serviceName: '',
- appId: '',
-
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Services",
- routeName: 'yarn-services',
- }],
-
- updateBreadcrumbs(appId, serviceName, componentName, instanceName) {
- var crumbs = [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Services",
- routeName: 'yarn-services',
- }];
- if (appId && serviceName && componentName && instanceName) {
- crumbs.push({
- text: `${serviceName} [${appId}]`,
- href: `#/yarn-app/${appId}/components?service=${serviceName}`
- }, {
- text: 'Components'
- }, {
- text: `${componentName}`,
- href: `#/yarn-component-instances/${componentName}/info?service=${serviceName}&&appid=${appId}`
- }, {
- text: `${instanceName}`
- });
- }
- this.set('breadcrumbs', crumbs);
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instance/info.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instance/info.js
deleted file mode 100644
index e920aa2e10443..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instance/info.js
+++ /dev/null
@@ -1,26 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- queryParams: ["appid", "service", "containerid"],
- appid: undefined,
- service: undefined,
- containerid: undefined
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instances.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instances.js
deleted file mode 100644
index fbd034ab78526..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instances.js
+++ /dev/null
@@ -1,58 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- componentName: '',
- serviceName: '',
- appId: '',
-
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Services",
- routeName: 'yarn-services',
- }],
-
- updateBreadcrumbs(appId, serviceName, componentName, tailCrumbs) {
- var crumbs = [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Services",
- routeName: 'yarn-services',
- }];
- if (appId && serviceName && componentName) {
- crumbs.push({
- text: `${serviceName} [${appId}]`,
- href: `#/yarn-app/${appId}/components?service=${serviceName}`
- }, {
- text: 'Components'
- }, {
- text: `${componentName}`,
- href: `#/yarn-component-instances/${componentName}/info?service=${serviceName}&&appid=${appId}`
- });
- }
- if (tailCrumbs) {
- crumbs.pushObjects(tailCrumbs);
- }
- this.set('breadcrumbs', crumbs);
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instances/configs.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instances/configs.js
deleted file mode 100644
index dac649851d7e3..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instances/configs.js
+++ /dev/null
@@ -1,25 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- queryParams: ["service", "appid"],
- appid: undefined,
- service: undefined
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instances/info.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instances/info.js
deleted file mode 100644
index 16e0e03c3e638..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-component-instances/info.js
+++ /dev/null
@@ -1,81 +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.
- */
-
-import Ember from 'ember';
-import ColumnDef from 'em-table/utils/column-definition';
-import TableDefinition from 'em-table/utils/table-definition';
-
-export default Ember.Controller.extend({
- queryParams: ["service", "appid"],
- appid: undefined,
- service: undefined,
-
- tableDefinition: TableDefinition.create({
- searchType: 'manual',
- }),
-
- tableColumns: Ember.computed('model.appId', 'model.serviceName', function() {
- var cols = [];
- var appId = this.get('model.appId');
- var serviceName = this.get('model.serviceName');
-
- cols.push({
- id: 'instanceName',
- headerTitle: 'Component Instance',
- contentPath: 'instanceName',
- cellComponentName: 'em-table-linked-cell',
- getCellContent: function(row) {
- var component = row.get('component');
- var instance = row.get('instanceName');
- var containerId = row.get('containerId');
- return {
- text: instance,
- href: `#/yarn-component-instance/${component}/instances/${instance}/info?appid=${appId}&service=${serviceName}&containerid=${containerId}`
- };
- }
- }, {
- id: 'containerId',
- headerTitle: 'Current Container Id',
- contentPath: 'containerId',
- minWidth: '350px'
- }, {
- id: 'state',
- headerTitle: 'State',
- contentPath: 'state'
- }, {
- id: 'startedDate',
- headerTitle: 'Started Time',
- contentPath: 'startedDate'
- }, {
- id: 'logsLink',
- headerTitle: 'Logs',
- contentPath: 'logsLink',
- cellComponentName: 'em-table-html-cell',
- getCellContent: function(row) {
- var containerLogUrl = row.get('containerLogURL');
- if (containerLogUrl) {
- return `Link`;
- } else {
- return 'N/A';
- }
- }
- });
-
- return ColumnDef.make(cols);
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-container-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-container-log.js
deleted file mode 100644
index d23b2b379f56f..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-container-log.js
+++ /dev/null
@@ -1,44 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
-
- breadcrumbs: Ember.computed('model.nodeInfo', 'model.containerInfo', function () {
- var nodeInfo = this.get('model.nodeInfo'),
- containerInfo = this.get('model.containerInfo');
- var nodeAddr = encodeURIComponent(nodeInfo.addr);
- return [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Nodes",
- routeName: 'yarn-nodes.table'
- }, {
- text: `Node [ ${nodeInfo.id} ]`,
- href: `#/yarn-node/${nodeInfo.id}/${nodeAddr}/info`,
- }, {
- text: `Container [ ${containerInfo.id} ]`,
- href: `#/yarn-node-container/${nodeInfo.id}/${nodeAddr}/${containerInfo.id}`,
- }, {
- text: "Log",
- }];
- })
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-deploy-service.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-deploy-service.js
deleted file mode 100644
index 38c84d690140c..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-deploy-service.js
+++ /dev/null
@@ -1,73 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Services",
- routeName: 'yarn-services',
- }, {
- text: "New Service",
- routeName: 'yarn-deploy-service',
- }],
-
- savedStandardTemplates: [],
- savedJsonTemplates: [],
- serviceResponse: null,
- isLoading: false,
-
- actions: {
- deployServiceDef(serviceDef, userName) {
- var defjson = serviceDef.getServiceJSON();
- this.deployServiceApp(defjson, userName);
- },
-
- deployServiceJson(json, userName) {
- this.deployServiceApp(json, userName);
- }
- },
-
- gotoServices() {
- Ember.run.later(this, function() {
- this.set('serviceResponse', null);
- this.transitionToRoute('yarn-services');
- }, 1000);
- },
-
- deployServiceApp(requestJson, userName) {
- var self = this;
- var adapter = this.store.adapterFor('yarn-servicedef');
- this.set('isLoading', true);
- adapter.deployService(requestJson, userName).then(function() {
- self.set('serviceResponse', {message: 'Service has been accepted successfully. Redirecting to services in a second.', type: 'success'});
- self.gotoServices();
- }, function(errr) {
- let messg = 'Error: Deploy service failed!';
- if (errr.errors && errr.errors[0] && errr.errors[0].diagnostics) {
- messg = 'Error: ' + errr.errors[0].diagnostics;
- }
- self.set('serviceResponse', {message: messg, type: 'error'});
- }).finally(function() {
- self.set('isLoading', false);
- });
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow-activity.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow-activity.js
deleted file mode 100644
index 61e7823740961..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow-activity.js
+++ /dev/null
@@ -1,125 +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.
- */
-
-import Ember from 'ember';
-import TableDef from 'em-table/utils/table-definition';
-import ColumnDef from 'em-table/utils/column-definition';
-import lodash from 'lodash/lodash';
-
-function _createColumns() {
- var columns = [];
- columns.push({
- id: 'flowName',
- headerTitle: 'Flow Name',
- contentPath: 'flowName',
- observePath: true,
- }, {
- id: 'user',
- headerTitle: 'User',
- contentPath: 'user',
- observePath: true
- }, {
- id: 'uid',
- headerTitle: 'Flow ID',
- contentPath: 'uid',
- observePath: true,
- cellComponentName: 'em-table-linked-cell',
- minWidth: "300px",
- getCellContent: function (row) {
- return {
- routeName: 'yarn-flow.info',
- id: row.get('uid'),
- displayText: row.get('uid')
- };
- }
- }, {
- id: 'lastExecDate',
- headerTitle: 'Last Execution Date',
- contentPath: 'lastExecDate',
- observePath: true
- });
- return ColumnDef.make(columns);
-}
-
-function _getAggregatedFlowsData(flows) {
- var aggregatedFlows = [];
- flows = flows? flows.get('content') : [];
-
- var aggregated = lodash.groupBy(flows, function(flow) {
- return flow.getRecord().get('uid');
- });
-
- lodash.forIn(aggregated, function(flows) {
- let flowsInAsc = lodash.sortBy(flows, function(flow) {
- return flow.getRecord().get('lastExecDate');
- });
- let flowsInDesc = flowsInAsc.reverse();
- aggregatedFlows.push(flowsInDesc[0].getRecord());
- });
-
- return aggregatedFlows;
-}
-
-function _createRows(flows) {
- var data = [],
- aggregatedFlows = null,
- row = null;
-
- aggregatedFlows = _getAggregatedFlowsData(flows);
-
- aggregatedFlows.forEach(function(flow) {
- row = Ember.Object.create({
- user: flow.get('user'),
- flowName: flow.get('flowName'),
- uid: flow.get('uid'),
- lastExecDate: flow.get('lastExecDate')
- });
- data.push(row);
- });
-
- return Ember.A(data);
-}
-
-export default Ember.Controller.extend({
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Flow Activities",
- routeName: 'yarn-flow-activity',
- }],
-
- columns: _createColumns(),
-
- rows: Ember.computed('model', function() {
- return _createRows(this.get('model'));
- }),
-
- tableDefinition: TableDef.create({
- sortColumnId: 'lastExecDate',
- sortOrder: 'desc'
- }),
-
- getLastFlowExecutionInfoByFlowUid: function(uid) {
- var aggregatedFlows = _getAggregatedFlowsData(this.get('model'));
- var recent = aggregatedFlows.find(function(flow) {
- return flow.get('uid') === uid;
- });
- return recent;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow.js
deleted file mode 100644
index 216ab1145d645..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow.js
+++ /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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- breadcrumbs: Ember.computed("model.flowUid", function() {
- var flowUid = this.get('model.flowUid');
- return [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Flow Activities",
- routeName: 'yarn-flow-activity'
- }, {
- text: `Flow Info [${flowUid}]`,
- routeName: 'yarn-flow.info',
- model: flowUid
- }];
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow/info.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow/info.js
deleted file mode 100644
index fc4dd2318bbad..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow/info.js
+++ /dev/null
@@ -1,66 +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.
- */
-
-import Ember from 'ember';
-import Converter from 'yarn-ui/utils/converter';
-
-export default Ember.Controller.extend({
- flowUid: function() {
- return this.get('model.flowUid');
- }.property('model.flowUid'),
-
- flowLastExecutionDate: function() {
- if (this.get('model.lastFlowExecutionInfo')) {
- return this.get('model.lastFlowExecutionInfo').get('lastExecDate');
- } else {
- return '';
- }
- }.property('model.lastFlowExecutionInfo'),
-
- flowInfo: function() {
- var info = {};
- var firstRunObj = this.get('model.flowRuns').get('firstObject');
- info.flowUid = this.get('flowUid');
- info.flowName = firstRunObj.get('flowName');
- info.user = firstRunObj.get('user');
- info.lastExecutionDate = this.get('flowLastExecutionDate');
- info.firstRunStarted = this.get('earliestStartTime');
- info.lastRunFinished = this.get('latestFinishTime');
- return info;
- }.property('model.flowRuns', 'flowLastExecutionDate'),
-
- earliestStartTime: function() {
- var earliestStart = Number.MAX_VALUE;
- this.get('model.flowRuns').forEach(function(flowrun) {
- if (flowrun.get('createTimeRaw') < earliestStart) {
- earliestStart = flowrun.get('createTimeRaw');
- }
- });
- return Converter.timeStampToDate(earliestStart);
- }.property('model.flowRuns'),
-
- latestFinishTime: function() {
- var latestFinish = 0;
- this.get('model.flowRuns').forEach(function(flowrun) {
- if (flowrun.get('endTimeRaw') > latestFinish) {
- latestFinish = flowrun.get('endTimeRaw');
- }
- });
- return Converter.timeStampToDate(latestFinish);
- }.property('model.flowRuns')
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow/runs.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow/runs.js
deleted file mode 100644
index a6c14628ddc56..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flow/runs.js
+++ /dev/null
@@ -1,178 +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.
- */
-
-import Ember from 'ember';
-import TableDef from 'em-table/utils/table-definition';
-import ColumnDef from 'em-table/utils/column-definition';
-import Converter from 'yarn-ui/utils/converter';
-
-function createColumn() {
- var columns = [];
-
- // Generate columns
- columns.push({
- id: 'runid',
- headerTitle: 'Run ID',
- contentPath: 'runid',
- cellComponentName: 'em-table-linked-cell',
- minWidth: "300px",
- getCellContent: function (row) {
- return {
- routeName: 'yarn-flowrun.info',
- id: row.get('uid'),
- displayText: row.get('shownid')
- };
- }
- });
-
- columns.push({
- id: 'runDurationTs',
- headerTitle: 'Run Duration',
- contentPath: 'runDurationTs',
- getCellContent: function(row) {
- return Converter.msToElapsedTimeUnit(row.get('runDurationTs'));
- }
- });
-
- columns.push({
- id: 'cpuVCores',
- headerTitle: 'CPU VCores',
- contentPath: 'cpuVCores',
- getCellContent: function(row) {
- if (row.get('cpuVCores') > -1) {
- return row.get('cpuVCores');
- }
- return 'N/A';
- }
- });
-
- columns.push({
- id: 'memoryUsed',
- headerTitle: 'Memory Used',
- contentPath: 'memoryUsed',
- getCellContent: function(row) {
- if (row.get('memoryUsed') > -1) {
- return Converter.memoryBytesToMB(row.get('memoryUsed'));
- }
- return 'N/A';
- }
- });
-
- columns.push({
- id: 'createTime',
- headerTitle: 'Creation Time',
- contentPath: 'createTime'
- });
-
- columns.push({
- id: 'endTime',
- headerTitle: 'End Time',
- contentPath: 'endTime'
- });
-
- return ColumnDef.make(columns);
-}
-
-export default Ember.Controller.extend({
- vizWidgets: {
- runDuration: true,
- cpuVcores: false,
- memoryUsed: false
- },
-
- actions: {
- addVizWidget(widget) {
- Ember.set(this.vizWidgets, widget, true);
- },
-
- removeVizWidget(widget) {
- Ember.set(this.vizWidgets, widget, false);
- }
- },
-
- columns: createColumn(),
-
- tableDefinition: TableDef.create({
- sortColumnId: 'createTime',
- sortOrder: 'desc'
- }),
-
- elapsedTimeVizData: function() {
- var data = [];
- this.get('model.flowRuns').forEach(function(run) {
- var vizData = run.getElapsedTimeVizDataForBarChart();
- if (vizData.value > 0) {
- data.push(vizData);
- }
- });
- data = this.getSortedVizDataInDesc(data);
- return this.getRefactoredVizData(data);
- }.property('model.flowRuns'),
-
- elapsedTimeFormatter: function(tick) {
- return Converter.msToElapsedTimeUnit(tick, true);
- },
-
- cpuVCoresVizData: function() {
- var data = [];
- this.get('model.flowRuns').forEach(function(run) {
- var vizData = run.getCpuVCoresVizDataForBarChart();
- if (vizData.value > 0) {
- data.push(vizData);
- }
- });
- data = this.getSortedVizDataInDesc(data);
- return this.getRefactoredVizData(data);
- }.property('model.flowRuns'),
-
- memoryVizData: function() {
- var data = [];
- this.get('model.flowRuns').forEach(function(run) {
- var vizData = run.getMemoryVizDataForBarChart();
- if (vizData.value > 0) {
- data.push(vizData);
- }
- });
- data = this.getSortedVizDataInDesc(data);
- return this.getRefactoredVizData(data);
- }.property('model.flowRuns'),
-
- memoryFormatter: function(tick) {
- return Converter.memoryBytesToMB(tick);
- },
-
- onBarChartClick: function() {
- var self = this;
- return function(data) {
- self.transitionToRoute('yarn-flowrun.info', data.flowrunUid);
- };
- }.property(),
-
- getSortedVizDataInDesc: function(data) {
- return data.sort(function(d1, d2) {
- return d2.createdTs - d1.createdTs;
- });
- },
-
- getRefactoredVizData: function(data) {
- data.forEach(function(viz, idx) {
- viz.label = "Run " + (++idx);
- }, this);
- return data;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flowrun.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flowrun.js
deleted file mode 100644
index 6af87ab1ee100..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flowrun.js
+++ /dev/null
@@ -1,50 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- breadcrumbs: Ember.computed('model.flowrun_uid', 'model.parentFlowUid', function() {
- var flowRunId = this.get('model.flowrun_uid');
- var parentFlowUid = this.get('model.parentFlowUid');
- var crumbs = [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Flow Activities",
- routeName: 'yarn-flow-activity'
- }];
- if (parentFlowUid) {
- crumbs.push({
- text: `Flow Info [${parentFlowUid}]`,
- routeName: 'yarn-flow.info',
- model: parentFlowUid
- }, {
- text: `Flow Runs [${parentFlowUid}]`,
- routeName: 'yarn-flow.runs',
- model: parentFlowUid
- });
- }
- crumbs.push({
- text: `Run Info [${flowRunId}]`,
- routeName: 'yarn-flowrun.info',
- model: flowRunId
- });
- return crumbs;
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flowrun/info.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flowrun/info.js
deleted file mode 100644
index b72afc4a77a09..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flowrun/info.js
+++ /dev/null
@@ -1,162 +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.
- */
-
-import Ember from 'ember';
-import ColumnDef from 'em-table/utils/column-definition';
-import Converter from 'yarn-ui/utils/converter';
-import TableDefinition from 'em-table/utils/table-definition';
-
-function createColumn() {
- var columns = [];
-
- tableDefinition: TableDefinition.create({
- searchType: 'manual',
- }),
-
- // Generate columns
- columns.push({
- id: 'appId',
- headerTitle: 'Application ID',
- contentPath: 'appId',
- cellComponentName: 'em-table-linked-cell',
- minWidth: "300px",
- getCellContent: function (row) {
- return {
- routeName: 'yarn-app.attempts',
- id: row.get('appId'),
- displayText: row.get('appId')
- };
- }
- });
-
- columns.push({
- id: 'appType',
- headerTitle: 'Application Type',
- contentPath: 'type'
- });
-
- columns.push({
- id: 'state',
- headerTitle: 'State',
- contentPath: 'state',
- cellComponentName: 'em-table-status-cell',
- });
-
- columns.push({
- id: 'elapsedTs',
- headerTitle: 'Elapsed Time',
- contentPath: 'elapsedTs',
- getCellContent: function(row) {
- return Converter.msToElapsedTimeUnit(row.get('elapsedTs'));
- }
- });
-
- columns.push({
- id: 'cpuVCores',
- headerTitle: 'CPU VCores',
- contentPath: 'cpuVCores',
- getCellContent: function(row) {
- if (row.get('cpuVCores') > -1) {
- return row.get('cpuVCores');
- }
- return 'N/A';
- }
- });
-
- columns.push({
- id: 'memoryUsed',
- headerTitle: 'Memory Used',
- contentPath: 'memoryUsed',
- getCellContent: function(row) {
- if (row.get('memoryUsed') > -1) {
- return Converter.memoryBytesToMB(row.get('memoryUsed'));
- }
- return 'N/A';
- }
- });
-
- return ColumnDef.make(columns);
-}
-
-export default Ember.Controller.extend({
- vizWidgets: {
- cpuVcores: true,
- memoryUsed: false
- },
-
- actions: {
- addVizWidget(widget) {
- Ember.set(this.vizWidgets, widget, true);
- },
-
- removeVizWidget(widget) {
- Ember.set(this.vizWidgets, widget, false);
- }
- },
-
- columns: createColumn(),
-
- cpuVCoresVizData: function() {
- var data = [];
- this.get('model.apps').forEach(function(app) {
- var vizData = app.getCpuVCoresVizDataForBarChart();
- if (vizData.value > 0) {
- data.push(vizData);
- }
- });
- data = this.getSortedVizDataInDesc(data);
- return this.getRefactoredVizData(data);
- }.property('model.apps'),
-
- memoryVizData: function() {
- var data = [];
- this.get('model.apps').forEach(function(app) {
- var vizData = app.getMemoryVizDataForBarChart();
- if (vizData.value > 0) {
- data.push(vizData);
- }
- });
- data = this.getSortedVizDataInDesc(data);
- return this.getRefactoredVizData(data);
- }.property('model.apps'),
-
- memoryFormatter: function(tick) {
- return Converter.memoryBytesToMB(tick);
- },
-
- onBarChartClick: function() {
- var self = this;
- return function(data) {
- self.transitionToRoute('yarn-app', data.appId);
- };
- }.property(),
-
- getSortedVizDataInDesc: function(data) {
- return data.sort(function(d1, d2) {
- return d2.value - d1.value;
- });
- },
-
- getRefactoredVizData: function(data) {
- data.forEach(function(viz, idx) {
- viz.appId = viz.label;
- viz.label = "App " + (++idx);
- }, this);
- return data;
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flowrun/metrics.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flowrun/metrics.js
deleted file mode 100644
index 12e3c491528d8..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-flowrun/metrics.js
+++ /dev/null
@@ -1,131 +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.
- */
-
-import Ember from 'ember';
-import ColumnDef from 'em-table/utils/column-definition';
-import TableDefinition from 'em-table/utils/table-definition';
-
-function _createColumns() {
- var columns = [];
-
- columns.push({
- id: 'name',
- headerTitle: 'Name',
- contentPath: 'name',
- observePath: true,
- cellComponentName: 'em-table-html-cell',
- getCellContent: function(row) {
- var plainName = row.name;
- if (plainName.indexOf('MAP:') > -1 || plainName.indexOf('REDUCE:') > -1) {
- plainName = plainName.substring(plainName.indexOf(':') + 1);
- }
- return `${plainName}`;
- }
- }, {
- id: 'value',
- headerTitle: 'Value',
- contentPath: 'value',
- observePath: true
- });
-
- return ColumnDef.make(columns);
-}
-
-export default Ember.Controller.extend({
- mapMetrics: null,
- reduceMetrics: null,
- generalMetrics: null,
-
- tableDefinition: TableDefinition.create({
- searchType: 'manual',
- }),
-
- columns: Ember.computed(function() {
- return _createColumns(this.get('model.flowrun_uid'));
- }),
-
- metricsObserver: Ember.observer('model.flowrun', function() {
- var metrics = this.get('model.flowrun.metrics');
- var mapConfigs = [],
- reduceConfigs = [],
- generalConfigs = [];
-
- metrics.forEach(function(metric) {
- let id = metric.id;
- if (id.startsWith('MAP:')) {
- mapConfigs.push(metric);
- } else if (id.startsWith('REDUCE:')) {
- reduceConfigs.push(metric);
- } else {
- generalConfigs.push(metric);
- }
- }, this);
-
- this.set('mapMetrics', mapConfigs);
- this.set('reduceMetrics', reduceConfigs);
- this.set('generalMetrics', generalConfigs);
- }),
-
- mapConfigRows: Ember.computed('mapMetrics', function() {
- var row = null,
- data = [];
-
- this.get('mapMetrics').forEach(function(map) {
- let value = map.values[Object.keys(map.values)[0]];
- row = Ember.Object.create({
- name: map.id,
- value: value
- });
- data.push(row);
- }, this);
-
- return Ember.A(data);
- }),
-
- reduceConfigRows: Ember.computed('reduceMetrics', function() {
- var row = null,
- data = [];
-
- this.get('reduceMetrics').forEach(function(map) {
- let value = map.values[Object.keys(map.values)[0]];
- row = Ember.Object.create({
- name: map.id,
- value: value
- });
- data.push(row);
- }, this);
-
- return Ember.A(data);
- }),
-
- generalConfigRows: Ember.computed('generalMetrics', function() {
- var row = null,
- data = [];
-
- this.get('generalMetrics').forEach(function(map) {
- let value = map.values[Object.keys(map.values)[0]];
- row = Ember.Object.create({
- name: map.id,
- value: value
- });
- data.push(row);
- }, this);
-
- return Ember.A(data);
- })
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-app.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-app.js
deleted file mode 100644
index 5ecf706f8f551..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-app.js
+++ /dev/null
@@ -1,45 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
-
- breadcrumbs: Ember.computed('model.nodeInfo', function () {
- var nodeInfo = this.get('model.nodeInfo');
- var addr = encodeURIComponent(nodeInfo.addr);
- return [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Nodes",
- routeName: 'yarn-nodes.table'
- }, {
- text: `Node [ ${nodeInfo.id} ]`,
- href: `#/yarn-node/${nodeInfo.id}/${addr}/info`,
- }, {
- text: `Application [ ${nodeInfo.appId} ]`,
- }];
- }),
-
- encodedNodeAddr: Ember.computed("model.nodeInfo", function() {
- var nodeInfo = this.get("model.nodeInfo");
- return encodeURIComponent(nodeInfo.addr);
- })
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-apps.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-apps.js
deleted file mode 100644
index 460f186af349e..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-apps.js
+++ /dev/null
@@ -1,45 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
-
- breadcrumbs: Ember.computed("model.attempt.appId", function () {
- var nodeInfo = this.get("model.nodeInfo");
- var addr = encodeURIComponent(nodeInfo.addr);
- return [{
- text: "Home",
- routeName: 'application'
- },{
- text: "Nodes",
- routeName: 'yarn-nodes.table'
- }, {
- text: `Node [ ${nodeInfo.id} ]`,
- href: `#/yarn-node/${nodeInfo.id}/${addr}/info`
- }, {
- text: "Applications",
- }];
- }),
-
- encodedNodeAddr: Ember.computed("model.nodeInfo", function() {
- var nodeInfo = this.get("model.nodeInfo");
- return encodeURIComponent(nodeInfo.addr);
- })
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-container.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-container.js
deleted file mode 100644
index bac26b2c91fff..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-container.js
+++ /dev/null
@@ -1,45 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
-
- breadcrumbs: Ember.computed("model.nodeInfo", function () {
- var nodeInfo = this.get("model.nodeInfo");
- var addr = encodeURIComponent(nodeInfo.addr);
- return [{
- text: "Home",
- routeName: 'application'
- },{
- text: "Nodes",
- routeName: 'yarn-nodes.table'
- }, {
- text: `Node [ ${nodeInfo.id} ]`,
- href: `#/yarn-node/${nodeInfo.id}/${addr}/info`
- }, {
- text: `Container [ ${nodeInfo.containerId} ]`
- }];
- }),
-
- encodedNodeAddr: Ember.computed("model.nodeInfo", function() {
- var nodeInfo = this.get("model.nodeInfo");
- return encodeURIComponent(nodeInfo.addr);
- })
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-containers.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-containers.js
deleted file mode 100644
index 4dbd05b2ff864..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node-containers.js
+++ /dev/null
@@ -1,45 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
-
- breadcrumbs: Ember.computed("model.nodeInfo", function () {
- var nodeInfo = this.get("model.nodeInfo");
- var addr = encodeURIComponent(nodeInfo.addr);
- return [{
- text: "Home",
- routeName: 'application'
- },{
- text: "Nodes",
- routeName: 'yarn-nodes.table'
- }, {
- text: `Node [ ${nodeInfo.id} ]`,
- href: `#/yarn-node/${nodeInfo.id}/${addr}/info`
- }, {
- text: "Containers",
- }];
- }),
-
- encodedNodeAddr: Ember.computed("model.nodeInfo", function() {
- var nodeInfo = this.get("model.nodeInfo");
- return encodeURIComponent(nodeInfo.addr);
- })
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node.js
deleted file mode 100644
index c6cf2e34940ab..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-node.js
+++ /dev/null
@@ -1,39 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
-
- breadcrumbs: Ember.computed("model.nodeInfo", function () {
- var nodeInfo = this.get("model.nodeInfo");
- var addr = encodeURIComponent(nodeInfo.addr);
-
- return [{
- text: "Home",
- routeName: 'application'
- },{
- text: "Nodes",
- routeName: 'yarn-nodes.table'
- }, {
- text: `Node [ ${nodeInfo.id} ]`,
- href: `#/yarn-node/${nodeInfo.id}/${addr}/info`,
- }];
- })
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes-heatmap.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes-heatmap.js
deleted file mode 100644
index a38d8c50f4366..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes-heatmap.js
+++ /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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- needReload: true,
- selectedQueue: undefined,
-
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Nodes",
- routeName: 'yarn-nodes.table',
- }, {
- text: "Heatmap",
- routeName: 'yarn-nodes-heatmap',
- }]
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes-status.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes-status.js
deleted file mode 100644
index b007cf4893888..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes-status.js
+++ /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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- needReload: true,
- selectedQueue: undefined,
-
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Nodes",
- routeName: 'yarn-nodes.table',
- }, {
- text: "Node status",
- routeName: 'yarn-nodes-status',
- }]
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes.js
deleted file mode 100644
index 24f9550dccd9c..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes.js
+++ /dev/null
@@ -1,33 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- needReload: true,
- selectedQueue: undefined,
-
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Nodes",
- routeName: 'yarn-nodes.table',
- }]
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes/table.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes/table.js
deleted file mode 100644
index 936e9358fdd5e..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-nodes/table.js
+++ /dev/null
@@ -1,137 +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.
- */
-
-
-import Ember from 'ember';
-import ColumnDef from 'em-table/utils/column-definition';
-import TableDefinition from 'em-table/utils/table-definition';
-
-export default Ember.Controller.extend({
- queryParams: ['searchText', 'sortColumnId', 'sortOrder', 'pageNum', 'rowCount'],
- tableDefinition: TableDefinition.create({
- searchType: 'manual',
- }),
- searchText: Ember.computed.alias('tableDefinition.searchText'),
- sortColumnId: Ember.computed.alias('tableDefinition.sortColumnId'),
- sortOrder: Ember.computed.alias('tableDefinition.sortOrder'),
- pageNum: Ember.computed.alias('tableDefinition.pageNum'),
- rowCount: Ember.computed.alias('tableDefinition.rowCount'),
- tableDefinition: TableDefinition.create({
- enableFaceting: true,
- minValuesToDisplay: 1,
- rowCount: 25
- }),
- columns: function() {
- var colums = [];
- colums.push({
- id: 'label',
- headerTitle: 'Node Label',
- contentPath: 'nodeLabelsAsString',
- minWidth: "125px"
- }, {
- id: 'rack',
- headerTitle: 'Rack',
- contentPath: 'rack',
- facetType: null,
- minWidth: "250px"
- }, {
- id: 'state',
- headerTitle: 'Node State',
- contentPath: 'state',
- cellComponentName: 'em-table-status-cell',
- minWidth: "125px"
- }, {
- id: 'address',
- headerTitle: 'Node Address',
- contentPath: 'id',
- minWidth: "300px",
- facetType: null,
- }, {
- id: 'nodeId',
- headerTitle: 'Node HTTP Address',
- contentPath: 'nodeHTTPAddress',
- cellComponentName: 'em-table-linked-cell',
- facetType: null,
- getCellContent: function(row) {
- var node_id = row.get("id"),
- node_addr = encodeURIComponent(row.get("nodeHTTPAddress")),
- href = `#/yarn-node/${node_id}/${node_addr}/info`;
- switch(row.get("nodeState")) {
- case "SHUTDOWN":
- case "LOST":
- href = "";
- }
-   return {
- text: row.get("nodeHTTPAddress"),
- href: href
- };
- },
- minWidth: "250px"
- }, {
- id: 'containers',
- headerTitle: 'Containers',
- contentPath: 'numContainers',
- facetType: null,
- }, {
- id: 'memUsed',
- headerTitle: 'Mem Used',
- contentPath: 'usedMemoryBytes',
- facetType: null,
- cellDefinition: {
- type: "memory"
- }
- }, {
- id: 'memAvail',
- headerTitle: 'Mem Available',
- contentPath: 'availMemoryBytes',
- facetType: null,
- cellDefinition: {
- type: "memory"
- }
- }, {
- id: 'coresUsed',
- headerTitle: 'VCores Used',
- contentPath: 'usedVirtualCores',
- facetType: null,
- }, {
- id: 'coresAvail',
- headerTitle: 'VCores Available',
- contentPath: 'availableVirtualCores',
- facetType: null,
- }, {
- id: 'healthUpdate',
- headerTitle: 'Last Health Update',
- contentPath: 'lastHealthUpdate',
- facetType: null,
- minWidth: "250px"
- }, {
- id: 'healthReport',
- headerTitle: 'Health-Report',
- contentPath: 'healthReport',
- facetType: null,
- minWidth: "200px"
- }, {
- id: 'version',
- headerTitle: 'Version',
- contentPath: 'version',
- facetType: null,
- observePath: true
- });
- return ColumnDef.make(colums);
- }.property()
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue.js
deleted file mode 100644
index e9f945ba722e6..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue.js
+++ /dev/null
@@ -1,49 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- needReload: true,
- selectedQueue: undefined,
-
- breadcrumbs: Ember.computed("model.selected", "target.currentPath", function () {
- var queueName = this.get("model.selected");
- var path = this.get("target.currentPath");
-
- var crumbs = [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Queues",
- routeName: 'yarn-queues',
- model: 'root'
- }];
-
- if (path && path === "yarn-queue.apps") {
- crumbs.push({
- text: `Queue [ ${queueName} ]`,
- routeName: 'yarn-queue.apps',
- model: queueName
- });
- }
-
- return crumbs;
- })
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue/apps.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue/apps.js
deleted file mode 100644
index 1ce630638a754..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queue/apps.js
+++ /dev/null
@@ -1,34 +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.
- */
-
-import Ember from 'ember';
-import TableDefinition from 'em-table/utils/table-definition';
-import AppTableController from '../app-table-columns';
-
-export default AppTableController.extend({
- tableDefinition: TableDefinition.create({
- searchType: 'manual',
- enableFaceting: true,
- rowCount: 25
- }),
-
- // Search text alias, any change in controller's searchText would affect the table's searchText, and vice-versa.
- _selectedObserver: Ember.on("init", Ember.observer("model.selected", function () {
- this.set("tableDefinition.searchText", this.get("model.selected"));
- })),
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js
deleted file mode 100644
index 6cc87675b50c6..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js
+++ /dev/null
@@ -1,55 +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.
- */
-
-import Ember from 'ember';
-import {PARTITION_LABEL} from '../constants';
-
-export default Ember.Controller.extend({
- needReload: true,
- selectedQueue: undefined,
- showLoading: true,
- filteredPartition: PARTITION_LABEL,
-
- breadcrumbs: [
- {
- text: "Home",
- routeName: "application"
- },
- {
- text: "Queues",
- routeName: "yarn-queues",
- model: "root"
- }
- ],
-
- actions: {
- setFilter(partition) {
- this.set("filteredPartition", partition);
- const model = this.get('model');
- const {selectedQueue} = model;
- // If the selected queue does not have the filtered partition
- // reset it to root
- if (!selectedQueue.get('partitions').contains(partition)) {
- const root = model.queues.get('firstObject');
- document.location.href = "#/yarn-queues/" + root.get("id") + "!";
- this.set("model.selectedQueue", root);
- this.set("model.selected", root.get('id'));
- }
- }
- }
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-services.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-services.js
deleted file mode 100644
index 1af915e0b3e4b..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-services.js
+++ /dev/null
@@ -1,115 +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.
- */
-
-import Ember from 'ember';
-import AppTableController from './app-table-columns';
-import TableDefinition from 'em-table/utils/table-definition';
-
-export default AppTableController.extend({
- queryParams: ['searchText', 'sortColumnId', 'sortOrder', 'pageNum', 'rowCount'],
- tableDefinition: TableDefinition.create({
- searchType: 'manual',
- sortColumnId: 'stTime',
- sortOrder: 'desc',
- rowCount: 25,
- minValuesToDisplay: 1,
- enableFaceting: true
- }),
- searchText: Ember.computed.alias('tableDefinition.searchText'),
- sortColumnId: Ember.computed.alias('tableDefinition.sortColumnId'),
- sortOrder: Ember.computed.alias('tableDefinition.sortOrder'),
- pageNum: Ember.computed.alias('tableDefinition.pageNum'),
- rowCount: Ember.computed.alias('tableDefinition.rowCount'),
-
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "Services",
- routeName: 'yarn-services',
- }],
-
- getFinishedServicesDataForDonutChart: Ember.computed('model.apps', function() {
-
- var finishdApps = 0;
- var failedApps = 0;
- var killedApps = 0;
-
- this.get('model.apps').forEach(function(service){
- if (service.get('state') === "FINISHED") {
- finishdApps++;
- }
-
- if (service.get('state') === "FAILED") {
- failedApps++;
- }
-
- if (service.get('state') === "KILLED") {
- killedApps++;
- }
- });
-
- var arr = [];
- arr.push({
- label: "Completed",
- value: finishdApps
- });
- arr.push({
- label: "Killed",
- value: killedApps
- });
- arr.push({
- label: "Failed",
- value: failedApps
- });
-
- return arr;
- }),
-
-
- getRunningServicesDataForDonutChart: Ember.computed('model.apps', function() {
- var pendingApps = 0;
- var runningApps = 0;
-
- this.get('model.apps').forEach(function(service){
- if (service.get('state') === "RUNNING") {
- runningApps++;
- }
-
- if (service.get('state') === "ACCEPTED" ||
- service.get('state') === "SUBMITTED" ||
- service.get('state') === "NEW" ||
- service.get('state') === "NEW_SAVING") {
- pendingApps++;
- }
- });
-
- var arr = [];
- arr.push({
- label: "Pending",
- value: pendingApps
- });
- arr.push({
- label: "Running",
- value: runningApps
- });
-
- return arr;
- }),
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools.js
deleted file mode 100644
index cb0c8d3a099a4..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools.js
+++ /dev/null
@@ -1,29 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- breadcrumbs: [{
- text: "Home",
- routeName: 'application'
- }, {
- text: "YARN Tools",
- routeName: 'yarn-tools',
- }],
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-conf.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-conf.js
deleted file mode 100644
index cc3be2e526f5d..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-conf.js
+++ /dev/null
@@ -1,60 +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.
- */
-
-import Ember from 'ember';
-
-import TableDef from 'em-table/utils/table-definition';
-import ColumnDef from 'em-table/utils/column-definition';
-
-import YarnConf from '../../models/yarn-conf';
-
-export default Ember.Controller.extend({
- coreTableDefinition: TableDef.create({
- searchType: 'manual',
- }),
-
- mapredTableDefinition: TableDef.create({
- searchType: 'manual',
- }),
-
- yarnTableDefinition: TableDef.create({
- searchType: 'manual',
- }),
-
- init: function () {
- var that = this;
- this.get('store').query('yarn-conf', {})
- .then(function(conf) {
- let coreProps = conf.filter(function(o) {
- return o.get('source') == 'core-default.xml';
- });
- that.set('rowsForCoreColumnsFromModel', coreProps);
- let mapredProps = conf.filter(function(o) {
- return o.get('source') == 'mapred-default.xml';
- });
- that.set('rowsForMapredColumnsFromModel', mapredProps);
- let yarnProps = conf.filter(function(o) {
- return o.get('source') == 'yarn-default.xml';
- });
- that.set('rowsForYarnColumnsFromModel', yarnProps);
- });
- },
-
- columnsFromModel: ColumnDef.makeFromModel(YarnConf),
-
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-rm-log.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-rm-log.js
deleted file mode 100644
index a5e0eb5744b42..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-tools/yarn-rm-log.js
+++ /dev/null
@@ -1,24 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Controller.extend({
- queryParams: ['filename'],
- filename: null
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/check-availability.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/check-availability.js
deleted file mode 100644
index 4470d6542a392..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/check-availability.js
+++ /dev/null
@@ -1,28 +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.
- */
-
-import Ember from 'ember';
-
-export function checkAvailability(params/*, hash*/) {
- if (params[0] !== undefined && params[0] !== null && params[0] !== '') {
- return params[0];
- }
- return 'N/A';
-}
-
-export default Ember.Helper.helper(checkAvailability);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/date-formatter.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/date-formatter.js
deleted file mode 100644
index 17834e4df9c18..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/date-formatter.js
+++ /dev/null
@@ -1,29 +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.
- */
-
-import Ember from 'ember';
-
-import Converter from 'yarn-ui/utils/converter';
-
-export function dateFormatter(params) {
- const [timestamp, dateOnly] = params;
-
- return dateOnly ? Converter.timeStampToDateOnly(timestamp) : Converter.timeStampToDate(timestamp);
-}
-
-export default Ember.Helper.helper(dateFormatter);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/divide.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/divide.js
deleted file mode 100644
index 437def8a12e6f..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/divide.js
+++ /dev/null
@@ -1,31 +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.
- */
-import Ember from 'ember';
-
-/**
- * Helper assumes values are numeric. num means numerator and
- * den means denominator.
- */
-export default Ember.Helper.helper(function(params,hash) {
- var num = hash.num;
- var den = hash.den;
- if (den === 0) {
- return 0;
- }
- return Math.floor(num/den);
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/json-pretty.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/json-pretty.js
deleted file mode 100644
index a820c38778a73..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/json-pretty.js
+++ /dev/null
@@ -1,25 +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.
- */
-import Ember from 'ember';
-
-export function jsonPretty(params/*, hash*/) {
- let j = params[0];
- return j;
-}
-
-export default Ember.Helper.helper(jsonPretty);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/log-adapter-helper.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/log-adapter-helper.js
deleted file mode 100644
index 616a38b3f1c74..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/log-adapter-helper.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-export function createEmptyContainerLogInfo(location) {
- return {
- containerLogsInfo: {
- containerLogInfo: [{
- fileName: "",
- fileSize: "",
- lastModifiedTime: "",
- redirectedUrl: location
- }]
- }
- };
-}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/log-files-comma.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/log-files-comma.js
deleted file mode 100644
index 963fc1418c1b7..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/log-files-comma.js
+++ /dev/null
@@ -1,55 +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.
- */
-
-import Ember from 'ember';
-
-/**
- * Represent log files as comma separated list.
- */
-export default Ember.Helper.helper(function(params,hash) {
- var logFiles = hash.logFiles;
- if (logFiles == null) {
- return "";
- }
- var logFilesLen = logFiles.length;
- if (logFilesLen === 0) {
- return "";
- }
- var nodeId = hash.nodeId;
- var nodeAddr = hash.nodeAddr;
- var containerId = hash.containerId;
- var html = '
';
- for (var i = 0; i < logFilesLen; i++) {
- var logFileName = "";
- if (logFiles[i]) {
- if (typeof logFiles[i] === "object" && logFiles[i].containerLogFiles) {
- logFileName = logFiles[i].containerLogFiles;
- } else if (typeof logFiles[i] === "string") {
- logFileName = logFiles[i];
- }
- }
- html = html + '' + logFileName +
- '';
- if (i !== logFilesLen - 1) {
- html = html + ", ";
- }
- }
- html = html + '
';
- return Ember.String.htmlSafe(html);
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/lower.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/lower.js
deleted file mode 100644
index 34c936f9df68c..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/lower.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-
-import Ember from 'ember';
-
-export function lower(params) {
- const string = params[0];
- if(!string) {
- return "";
- }
- return string.toLowerCase();
-}
-
-export default Ember.Helper.helper(lower);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/node-link.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/node-link.js
deleted file mode 100644
index 4f412da8a830d..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/node-link.js
+++ /dev/null
@@ -1,37 +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.
- */
-
-import Ember from 'ember';
-
-/**
- * Generate link to node page if its not SHUTDOWN or LOST.
- */
-export default Ember.Helper.helper(function(params,hash) {
- var nodeState = hash.nodeState;
- var nodeHTTPAddress = hash.nodeHTTPAddress;
- var nodeId = hash.nodeId;
- var html = '
';
- if (nodeState === "SHUTDOWN" || nodeState === "LOST") {
- html = html + nodeHTTPAddress;
- } else {
- html = html + '' +
- nodeHTTPAddress + '';
- }
- html = html + '
';
- return Ember.String.htmlSafe(html);
-});
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/node-menu.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/node-menu.js
deleted file mode 100644
index ace50e652ff42..0000000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/node-menu.js
+++ /dev/null
@@ -1,66 +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.
- */
-import Ember from 'ember';
-
-/**
- * Create left hand side node manager menu with menu item activated based
- * on page being accessed.
- */
-export default Ember.Helper.helper(function(params,hash) {
- // Place a menu within a panel inside col-md-2 container.
- var nodeIdSplitAtPort = hash.nodeId;
- var portIndex = nodeIdSplitAtPort.indexOf(':');
- if (portIndex !== -1) {
- nodeIdSplitAtPort = nodeIdSplitAtPort.substring(0, portIndex) +
- ':' + nodeIdSplitAtPort.substring(portIndex + 1);
- }
- var normalizedNodeId = '';
- var splitsAlongDots = nodeIdSplitAtPort.split('.');
- if (splitsAlongDots) {
- var len = splitsAlongDots.length;
- for (var i = 0; i < len; i++) {
- normalizedNodeId = normalizedNodeId + splitsAlongDots[i];
- if (i !== len - 1) {
- normalizedNodeId = normalizedNodeId + '.';
- }
- }
- } else {
- normalizedNodeId = nodeIdSplitAtPort;
- }
-
- var html = '