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.size 18 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

"; - } - else if (selectedOption === this.cpuLabel) { - return "

Used VCores: " + Math.round(data.get("usedVirtualCores")) + " VCores

" + - "

Available VCores: " + Math.round(data.get("availableVirtualCores")) + " VCores

"; - } - else if (selectedOption === this.containersLabel) { - return "

Containers: " + Math.round(data.get("numContainers")) + " Containers

" + - "

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 = 'Queues' + - '(current)Applications' + - '(current)Cluster Overview(current)Nodes' + - '(current)'; - 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 = '
'+ - '

Node Manager
(' + normalizedNodeId + ')

'+ - ''; - return Ember.String.htmlSafe(html); -}); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/prepend-protocol.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/prepend-protocol.js deleted file mode 100644 index e8d18c4472092..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/helpers/prepend-protocol.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 function prependProtocol(params/*, hash*/) { - let address = params[0]; - if (address && address.indexOf('://') < 0) { - address = 'http://' + address; - } - return address; -} - -export default Ember.Helper.helper(prependProtocol); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/index.html b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/index.html deleted file mode 100644 index 9fcf0eed408a7..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/index.html +++ /dev/null @@ -1,44 +0,0 @@ - - - - - - - - YARN - - - - {{content-for 'head'}} - - - - - {{content-for 'head-footer'}} - - - {{content-for 'body'}} - - - - - - {{content-for 'body-footer'}} - - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/env.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/env.js deleted file mode 100644 index bb9329ecf48c4..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/env.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 initialize( application ) { - application.inject('controller', 'env', 'service:env'); - application.inject('route', 'env', 'service:env'); - application.inject('adapter', 'env', 'service:env'); - application.inject('model', 'env', 'service:env'); -} - -export default { - name: 'env', - initialize -}; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/hosts.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/hosts.js deleted file mode 100644 index 6e3aa964c89b7..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/hosts.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. - */ - -export function initialize( application ) { - application.inject('controller', 'hosts', 'service:hosts'); - application.inject('route', 'hosts', 'service:hosts'); - application.inject('adapter', 'hosts', 'service:hosts'); -} - -export default { - name: 'hosts', - initialize -}; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/jquery.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/jquery.js deleted file mode 100644 index 9633cbcdedd32..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/jquery.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 Ember from 'ember'; - -export function initialize(/* application */) { - Ember.$(document).tooltip({ - tooltipClass: 'generic-tooltip', - selector: ".yarn-tooltip" - }); - - Ember.$.ajaxSetup({ - cache: false - }); -} - -export default { - name: 'jquery', - initialize -}; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js deleted file mode 100644 index bf544ecd51ec4..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js +++ /dev/null @@ -1,299 +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. - */ - -/* globals ENV: true */ - -import Ember from 'ember'; - -function getConfigFromYarn(rmhost, application, config) { - var httpUrl = window.location.protocol + '//' + - (ENV.hosts.localBaseAddress? ENV.hosts.localBaseAddress + '/' : '') + rmhost; - - httpUrl += '/conf?name=' + config; - Ember.Logger.log("The RM URL is: " + httpUrl); - - var configValue = ""; - $.ajax({ - type: 'GET', - dataType: 'json', - async: false, - context: this, - url: httpUrl, - success: function(data) { - configValue = data.property.value; - Ember.Logger.log("Value of the config returned from RM: " + configValue); - - application.advanceReadiness(); - }, - error: function() { - application.advanceReadiness(); - } - }); - return configValue; -} - -function getJHSURL(rmhost, application, isHttpsSchemeEnabled) { - Ember.Logger.log("getJHSURL, params:rmhost=" + rmhost + ",application=" + application + ",isHttpsSchemeEnabled=" + isHttpsSchemeEnabled); - var config = ''; - if (isHttpsSchemeEnabled) { - config = 'mapreduce.jobhistory.webapp.https.address'; - } else { - config = 'mapreduce.jobhistory.webapp.address'; - } - return getConfigFromYarn(rmhost, application, config); -} - -function getYarnHttpProtocolScheme(rmhost, application) { - return getConfigFromYarn(rmhost, application, 'yarn.http.policy'); -} - -function getTimeLineURL(rmhost, isHttpsSchemeEnabled) { - var url = window.location.protocol + '//' + - (ENV.hosts.localBaseAddress? ENV.hosts.localBaseAddress + '/' : '') + rmhost; - - if(isHttpsSchemeEnabled) { - url += '/conf?name=yarn.timeline-service.reader.webapp.https.address'; - } else { - url += '/conf?name=yarn.timeline-service.reader.webapp.address'; - } - - Ember.Logger.log("Get Timeline V2 Address URL: " + url); - return url; -} - -function getTimeLineV1URL(rmhost, isHttpsSchemeEnabled) { - var url = window.location.protocol + '//' + - (ENV.hosts.localBaseAddress? ENV.hosts.localBaseAddress + '/' : '') + rmhost; - - if(isHttpsSchemeEnabled) { - url += '/conf?name=yarn.timeline-service.webapp.https.address'; - } else { - url += '/conf?name=yarn.timeline-service.webapp.address'; - } - - Ember.Logger.log("Get Timeline V1 Address URL: " + url); - return url; -} - -function getSecurityURL(rmhost) { - var url = window.location.protocol + '//' + - (ENV.hosts.localBaseAddress? ENV.hosts.localBaseAddress + '/' : '') + rmhost; - - url += '/conf?name=hadoop.security.authentication'; - Ember.Logger.log("Server security mode url is: " + url); - return url; -} - -function getClusterIdFromYARN(rmhost, application) { - var httpUrl = window.location.protocol + '//' + - (ENV.hosts.localBaseAddress? ENV.hosts.localBaseAddress + '/' : '') + rmhost; - - httpUrl += '/conf?name=yarn.resourcemanager.cluster-id'; - Ember.Logger.log("Get cluster-id URL is: " + httpUrl); - - var clusterId = ""; - $.ajax({ - type: 'GET', - dataType: 'json', - async: false, - context: this, - url: httpUrl, - success: function(data) { - clusterId = data.property.value; - Ember.Logger.log("Cluster Id from RM: " + clusterId); - application.advanceReadiness(); - }, - error: function() { - application.advanceReadiness(); - } - }); - return clusterId; -} - -function getNodeManagerPort(rmhost, application) { - var httpUrl = window.location.protocol + "//" + - (ENV.hosts.localBaseAddress ? ENV.hosts.localBaseAddress + '/' : '') + rmhost - + "/conf?name=yarn.nodemanager.webapp.address"; - - var port = "8042"; - $.ajax({ - type: 'GET', - dataType: 'json', - async: false, - context: this, - url: httpUrl, - success: function(data) { - port = data.property.value.split(":")[1]; - application.advanceReadiness(); - }, - error: function() { - port = "8042"; - application.advanceReadiness(); - } - }); - return port; -} - -function transformURL(url, hostname) { - // Deleting the scheme from the beginning of the url - url = url.replace(/(^\w+:|^)\/\//, ''); - - var address = url.split(":")[0]; - var port = url.split(":")[1]; - // Instead of localhost, use the name of the host - if (address === "0.0.0.0" || address === "localhost") { - url = hostname + ":" + port; - } - - Ember.Logger.log("The transformed URL is: " + url); - return url; -} - -function updateConfigs(application) { - var hostname = window.location.hostname; - var rmhost = hostname + (window.location.port ? ':' + window.location.port: '') + - skipTrailingSlash(window.location.pathname); - - window.ENV = window.ENV || {}; - window.ENV.hosts = window.ENV.hosts || {}; - - if(!ENV.hosts.rmWebAddress) { - ENV.hosts.rmWebAddress = rmhost; - ENV.hosts.protocolScheme = window.location.protocol; - } else { - rmhost = ENV.hosts.rmWebAddress; - } - - Ember.Logger.log("RM Address: " + rmhost); - - var protocolSchemeFromRM = getYarnHttpProtocolScheme(rmhost, application); - Ember.Logger.log("Is protocol scheme https? " + (protocolSchemeFromRM == "HTTPS_ONLY")); - var isHttpsSchemeEnabled = (protocolSchemeFromRM == "HTTPS_ONLY"); - - var clusterIdFromYARN = getClusterIdFromYARN(rmhost, application); - ENV.clusterId = clusterIdFromYARN; - - var nodeManagerPort = getNodeManagerPort(rmhost, application); - Ember.Logger.log("NodeMananger port: " + nodeManagerPort); - ENV.nodeManagerPort = nodeManagerPort; - - if (!ENV.hosts.jhsAddress) { - var jhsAddress = getJHSURL(rmhost, application, isHttpsSchemeEnabled); - jhsAddress = transformURL(jhsAddress, hostname); - Ember.Logger.log("The JHS address is " + jhsAddress); - ENV.hosts.jhsAddress = jhsAddress; - } - - if(!ENV.hosts.timelineWebAddress) { - var timelinehost = ""; - $.ajax({ - type: 'GET', - dataType: 'json', - async: false, - context: this, - url: getTimeLineURL(rmhost, isHttpsSchemeEnabled), - success: function(data) { - timelinehost = data.property.value; - timelinehost = transformURL(timelinehost, hostname); - ENV.hosts.timelineWebAddress = timelinehost; - Ember.Logger.log("Timeline Address from RM: " + timelinehost); - - application.advanceReadiness(); - }, - error: function() { - application.advanceReadiness(); - } - }); - } else { - Ember.Logger.log("Timeline Address: " + ENV.hosts.timelineWebAddress); - application.advanceReadiness(); - } - - if(!ENV.hosts.timelineV1WebAddress) { - var timelinehost = ""; - $.ajax({ - type: 'GET', - dataType: 'json', - async: false, - context: this, - url: getTimeLineV1URL(rmhost, isHttpsSchemeEnabled), - success: function(data) { - timelinehost = data.property.value; - timelinehost = transformURL(timelinehost, hostname); - ENV.hosts.timelineV1WebAddress = timelinehost; - Ember.Logger.log("Timeline V1 Address from RM: " + timelinehost); - - application.advanceReadiness(); - }, - error: function() { - application.advanceReadiness(); - } - }); - } else { - Ember.Logger.log("Timeline V1 Address: " + ENV.hosts.timelineV1WebAddress); - application.advanceReadiness(); - } - - if(!ENV.hosts.isSecurityEnabled) { - var isSecurityEnabled = ""; - $.ajax({ - type: 'GET', - dataType: 'json', - async: false, - context: this, - url: getSecurityURL(rmhost), - success: function(data) { - isSecurityEnabled = data.property.value; - ENV.hosts.isSecurityEnabled = isSecurityEnabled; - Ember.Logger.log("Security mode is : " + isSecurityEnabled); - application.advanceReadiness(); - }, - error: function() { - application.advanceReadiness(); - } - }); - } else { - Ember.Logger.log("Security mode is: " + ENV.hosts.isSecurityEnabled); - application.advanceReadiness(); - } -} - -export function initialize( application ) { - application.deferReadiness(); - updateConfigs(application); -} - -export default { - name: 'loader', - before: 'env', - initialize -}; - -const skipTrailingSlash = function(path) { - path = path.replace('index.html', ''); - path = path.replace('ui2/', ''); - path = path.replace(/\/$/, ''); - console.log('base url:' + path) - if(path.includes("redirect")) { - var to = path.lastIndexOf('/'); - to = to == -1 ? path.length : to + 1; - path = path.substring(0, to); - console.log('base url after redirect:' + path) - } - return path; -}; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/mixins/app-attempt.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/mixins/app-attempt.js deleted file mode 100644 index a90bf36fd9aaf..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/mixins/app-attempt.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'; - -export default Ember.Mixin.create({ - fetchAppInfoFromRMorATS(appId, store) { - return new Ember.RSVP.Promise(function(resolve, reject) { - store.find('yarn-app', appId).then(function(rmApp) { - resolve(rmApp); - }, function() { - store.find('yarn-app-timeline', appId).then(function(atsApp) { - resolve(atsApp); - }, function() { - console.error('Error:', 'Application not found in RM or ATS for appId: ' + appId); - reject(null); - }); - }); - }); - }, - - fetchAttemptInfoFromRMorATS(attemptId, store) { - return new Ember.RSVP.Promise(function(resolve, reject) { - store.findRecord('yarn-app-attempt', attemptId, {reload: true}).then(function(rmAttempt) { - resolve(rmAttempt); - }, function() { - store.findRecord('yarn-timeline-appattempt', attemptId, {reload: true}).then(function(atsAttempt) { - resolve(atsAttempt); - }, function() { - console.error('Error:', 'Application attempt not found in RM or ATS for attemptId: ' + attemptId); - reject(null); - }); - }); - }); - }, - - fetchAttemptListFromRMorATS(appId, store) { - return new Ember.RSVP.Promise(function(resolve, reject) { - store.query('yarn-app-attempt', {appId: appId}).then(function(rmAttempts) { - resolve(rmAttempts); - }, function() { - store.query('yarn-timeline-appattempt', {appId: appId}).then(function(atsAttempts) { - resolve(atsAttempts); - }, function() { - console.error('Error:', 'Application attempts not found in RM or ATS for appId: ' + appId); - reject(null); - }); - }); - }); - } -}); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-info.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-info.js deleted file mode 100644 index c1a095a7740e1..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-info.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 DS from 'ember-data'; - -export default DS.Model.extend({ - startedOn: DS.attr('string'), - state: DS.attr('string'), - haState: DS.attr('string'), - rmStateStoreName: DS.attr('string'), - resourceManagerVersion: DS.attr('string'), - resourceManagerBuildVersion: DS.attr('string'), - hadoopVersion: DS.attr('string'), - hadoopBuildVersion: DS.attr('string'), - hadoopVersionBuiltOn: DS.attr('string'), - getYARNBuildHash: function() { - return this.get("hadoopVersion") + " from " + this.get("resourceManagerBuildVersion").split(" ")[2]; - }.property("yarnHash") -}); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-metric.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-metric.js deleted file mode 100644 index c1459b8ac8068..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/cluster-metric.js +++ /dev/null @@ -1,211 +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'; - -export default DS.Model.extend({ - appsSubmitted: DS.attr('number'), - appsCompleted: DS.attr('number'), - appsPending: DS.attr('number'), - appsRunning: DS.attr('number'), - appsFailed: DS.attr('number'), - appsKilled: DS.attr('number'), - reservedMB: DS.attr('number'), - availableMB: DS.attr('number'), - allocatedMB: DS.attr('number'), - reservedVirtualCores: DS.attr('number'), - availableVirtualCores: DS.attr('number'), - allocatedVirtualCores: DS.attr('number'), - containersAllocated: DS.attr('number'), - containersReserved: DS.attr('number'), - containersPending: DS.attr('number'), - totalMB: DS.attr('number'), - totalVirtualCores: DS.attr('number'), - totalNodes: DS.attr('number'), - lostNodes: DS.attr('number'), - unhealthyNodes: DS.attr('number'), - decommissioningNodes: DS.attr('number'), - decommissionedNodes: DS.attr('number'), - rebootedNodes: DS.attr('number'), - activeNodes: DS.attr('number'), - totalUsedResourcesAcrossPartition: DS.attr('object'), - totalClusterResourcesAcrossPartition: DS.attr('object'), - - getFinishedAppsDataForDonutChart: function() { - var arr = []; - arr.push({ - label: "Completed", - value: this.get("appsCompleted") - }); - arr.push({ - label: "Killed", - value: this.get("appsKilled") - }); - arr.push({ - label: "Failed", - value: this.get("appsFailed") - }); - - return arr; - }.property("appsCompleted", "appsKilled", "appsFailed"), - - getRunningAppsDataForDonutChart: function() { - var arr = []; - - arr.push({ - label: "Pending", - value: this.get("appsPending") - }); - arr.push({ - label: "Running", - value: this.get("appsRunning") - }); - - return arr; - }.property("appsPending", "appsRunning"), - - getNodesDataForDonutChart: function() { - var arr = []; - arr.push({ - label: "Active", - value: this.get("activeNodes") - }); - arr.push({ - label: "Unhealthy", - value: this.get("unhealthyNodes") - }); - arr.push({ - label: "Decommissioning", - value: this.get("decommissioningNodes") || 0 - }); - arr.push({ - label: "Decommissioned", - value: this.get("decommissionedNodes") - }); - arr.push({ - label: "Lost", - value: this.get("lostNodes") - }); - return arr; - }.property("activeNodes", "unhealthyNodes", "decommissioningNodes", "decommissionedNodes", "lostNodes"), - - getMemoryDataForDonutChart: function() { - var type = "MB"; - var arr = []; - arr.push({ - label: "Allocated", - value: this.get("allocated" + type) - }); - arr.push({ - label: "Reserved", - value: this.get("reserved" + type) - }); - arr.push({ - label: "Available", - value: this.get("available" + type) - }); - - return arr; - }.property("allocatedMB", "reservedMB", "availableMB"), - - getVCoreDataForDonutChart: function() { - var type = "VirtualCores"; - var arr = []; - arr.push({ - label: "Allocated", - value: this.get("allocated" + type) - }); - arr.push({ - label: "Reserved", - value: this.get("reserved" + type) - }); - arr.push({ - label: "Available", - value: Math.max(this.get("available" + type), 0) - }); - - return arr; - }.property("allocatedVirtualCores", "reservedVirtualCores", "availableVirtualCores"), - - getResourceTypes: function() { - var types = []; - if (this.get("totalClusterResourcesAcrossPartition")) { - - console.log(types); - } - }.property("totalClusterResourcesAcrossPartition"), - - /* - * Returned format - * [ - * { - * name: - * unit: - * [ - * { - * label: