You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2018/08/03 20:46:22 UTC

[33/36] storm git commit: STORM-1311: UI Migration from clj to java

http://git-wip-us.apache.org/repos/asf/storm/blob/034ac677/storm-core/src/jvm/org/apache/storm/command/Rebalance.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/Rebalance.java b/storm-core/src/jvm/org/apache/storm/command/Rebalance.java
index 4ecf4c9..2ca7630 100644
--- a/storm-core/src/jvm/org/apache/storm/command/Rebalance.java
+++ b/storm-core/src/jvm/org/apache/storm/command/Rebalance.java
@@ -12,6 +12,7 @@
 
 package org.apache.storm.command;
 
+import static java.lang.String.format;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.storm.generated.Nimbus;
@@ -22,7 +23,6 @@ import org.json.simple.JSONValue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.lang.String.format;
 
 public class Rebalance {
 
@@ -42,7 +42,6 @@ public class Rebalance {
         Integer numWorkers = (Integer) cl.get("n");
         Map<String, Integer> numExecutors = (Map<String, Integer>) cl.get("e");
         Map<String, Map<String, Double>> resourceOverrides = (Map<String, Map<String, Double>>) cl.get("r");
-        Map<String, Object> confOverrides = (Map<String, Object>) cl.get("t");
 
         if (null != wait) {
             rebalanceOptions.set_wait_secs(wait);
@@ -58,6 +57,8 @@ public class Rebalance {
             rebalanceOptions.set_topology_resources_overrides(resourceOverrides);
         }
 
+        Map<String, Object> confOverrides = (Map<String, Object>) cl.get("t");
+
         if (null != confOverrides) {
             rebalanceOptions.set_topology_conf_overrides(JSONValue.toJSONString(confOverrides));
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/034ac677/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java b/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java
index 24f04e1..7e9815a 100644
--- a/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java
+++ b/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java
@@ -65,10 +65,8 @@ public class SetLogLevel {
 
     /**
      * Parses [logger name]=[level string]:[optional timeout],[logger name2]...
-     *
      * e.g. ROOT=DEBUG:30
      *     root logger, debug for 30 seconds
-     *
      *     org.apache.foo=WARN
      *     org.apache.foo set to WARN indefinitely
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/034ac677/storm-core/src/jvm/org/apache/storm/command/UploadCredentials.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/UploadCredentials.java b/storm-core/src/jvm/org/apache/storm/command/UploadCredentials.java
index 101f8f2..2afebc0 100644
--- a/storm-core/src/jvm/org/apache/storm/command/UploadCredentials.java
+++ b/storm-core/src/jvm/org/apache/storm/command/UploadCredentials.java
@@ -34,6 +34,11 @@ public class UploadCredentials {
 
     private static final Logger LOG = LoggerFactory.getLogger(UploadCredentials.class);
 
+    /**
+     * Uploads credentials for a topology.
+     * @param args To accept topology name.
+     * @throws Exception on errors.
+     */
     public static void main(String[] args) throws Exception {
         Map<String, Object> cl = CLI.opt("f", "file", null)
                                     .arg("topologyName", CLI.FIRST_WINS)

http://git-wip-us.apache.org/repos/asf/storm/blob/034ac677/storm-core/src/jvm/org/apache/storm/ui/FilterConfiguration.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/ui/FilterConfiguration.java b/storm-core/src/jvm/org/apache/storm/ui/FilterConfiguration.java
deleted file mode 100644
index 153826e..0000000
--- a/storm-core/src/jvm/org/apache/storm/ui/FilterConfiguration.java
+++ /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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.ui;
-
-import java.util.Map;
-
-public class FilterConfiguration {
-    private String filterClass;
-    private String filterName;
-    private Map<String, String> filterParams;
-
-
-    public FilterConfiguration(String filterClass, Map<String, String> filterParams) {
-        this.filterParams = filterParams;
-        this.filterClass = filterClass;
-        this.filterName = null;
-    }
-
-    public FilterConfiguration(String filterClass, String filterName, Map<String, String> filterParams) {
-        this.filterClass = filterClass;
-        this.filterName = filterName;
-        this.filterParams = filterParams;
-    }
-
-    public String getFilterName() {
-        return filterName;
-    }
-
-    public void setFilterName(String filterName) {
-        this.filterName = filterName;
-    }
-
-    public String getFilterClass() {
-        return filterClass;
-    }
-
-    public void setFilterClass(String filterClass) {
-        this.filterClass = filterClass;
-    }
-
-    public Map<String, String> getFilterParams() {
-        return filterParams;
-    }
-
-    public void setFilterParams(Map<String, String> filterParams) {
-        this.filterParams = filterParams;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/034ac677/storm-core/src/jvm/org/apache/storm/ui/IConfigurator.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/ui/IConfigurator.java b/storm-core/src/jvm/org/apache/storm/ui/IConfigurator.java
deleted file mode 100644
index 9424677..0000000
--- a/storm-core/src/jvm/org/apache/storm/ui/IConfigurator.java
+++ /dev/null
@@ -1,19 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.ui;
-
-import org.eclipse.jetty.server.Server;
-
-public interface IConfigurator {
-    void execute(Server s);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/034ac677/storm-core/src/jvm/org/apache/storm/ui/InvalidRequestException.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/ui/InvalidRequestException.java b/storm-core/src/jvm/org/apache/storm/ui/InvalidRequestException.java
deleted file mode 100644
index 04f0596..0000000
--- a/storm-core/src/jvm/org/apache/storm/ui/InvalidRequestException.java
+++ /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.
- */
-
-package org.apache.storm.ui;
-
-public class InvalidRequestException extends Exception {
-
-    public InvalidRequestException() {
-        super();
-    }
-
-    public InvalidRequestException(String msg) {
-        super(msg);
-    }
-
-    public InvalidRequestException(String msg, Throwable cause) {
-        super(msg, cause);
-    }
-
-    public InvalidRequestException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/034ac677/storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java b/storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java
deleted file mode 100644
index af8999b..0000000
--- a/storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
- * and limitations under the License.
- */
-
-package org.apache.storm.ui;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableMap;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.net.URLEncoder;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import javax.servlet.DispatcherType;
-import javax.servlet.Servlet;
-import org.apache.storm.generated.ExecutorInfo;
-import org.apache.storm.logging.filters.AccessLoggingFilter;
-import org.apache.storm.utils.ObjectReader;
-import org.eclipse.jetty.http.HttpStatus;
-import org.eclipse.jetty.http.HttpVersion;
-import org.eclipse.jetty.server.HttpConfiguration;
-import org.eclipse.jetty.server.HttpConnectionFactory;
-import org.eclipse.jetty.server.SecureRequestCustomizer;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.server.SslConnectionFactory;
-import org.eclipse.jetty.servlet.FilterHolder;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.ServletHolder;
-import org.eclipse.jetty.servlets.CrossOriginFilter;
-import org.eclipse.jetty.util.ssl.SslContextFactory;
-import org.json.simple.JSONValue;
-
-public class UIHelpers {
-
-    private static final Object[][] PRETTY_SEC_DIVIDERS = {
-        new Object[]{ "s", 60 },
-        new Object[]{ "m", 60 },
-        new Object[]{ "h", 24 },
-        new Object[]{ "d", null }
-    };
-
-    private static final Object[][] PRETTY_MS_DIVIDERS = {
-        new Object[]{ "ms", 1000 },
-        new Object[]{ "s", 60 },
-        new Object[]{ "m", 60 },
-        new Object[]{ "h", 24 },
-        new Object[]{ "d", null }
-    };
-
-    public static String prettyUptimeStr(String val, Object[][] dividers) {
-        int uptime = Integer.parseInt(val);
-        LinkedList<String> tmp = new LinkedList<>();
-        for (Object[] divider : dividers) {
-            if (uptime > 0) {
-                String state = (String) divider[0];
-                Integer div = (Integer) divider[1];
-                if (div != null) {
-                    tmp.addFirst(uptime % div + state);
-                    uptime = uptime / div;
-                } else {
-                    tmp.addFirst(uptime + state);
-                }
-            }
-        }
-        return Joiner.on(" ").join(tmp);
-    }
-
-    public static String prettyUptimeSec(String sec) {
-        return prettyUptimeStr(sec, PRETTY_SEC_DIVIDERS);
-    }
-
-    public static String prettyUptimeSec(int secs) {
-        return prettyUptimeStr(String.valueOf(secs), PRETTY_SEC_DIVIDERS);
-    }
-
-    public static String prettyUptimeMs(String ms) {
-        return prettyUptimeStr(ms, PRETTY_MS_DIVIDERS);
-    }
-
-    public static String prettyUptimeMs(int ms) {
-        return prettyUptimeStr(String.valueOf(ms), PRETTY_MS_DIVIDERS);
-    }
-
-
-    public static String urlFormat(String fmt, Object... args) {
-        String[] argsEncoded = new String[args.length];
-        for (int i = 0; i < args.length; i++) {
-            argsEncoded[i] = URLEncoder.encode(String.valueOf(args[i]));
-        }
-        return String.format(fmt, argsEncoded);
-    }
-
-    public static String prettyExecutorInfo(ExecutorInfo e) {
-        return "[" + e.get_task_start() + "-" + e.get_task_end() + "]";
-    }
-
-    public static Map<String, Object> unauthorizedUserJson(String user) {
-        return ImmutableMap.of(
-            "error", "No Authorization",
-            "errorMessage", String.format("User %s is not authorized.", user));
-    }
-
-    private static ServerConnector mkSslConnector(Server server, Integer port, String ksPath, String ksPassword, String ksType,
-                                                  String keyPassword, String tsPath, String tsPassword, String tsType,
-                                                  Boolean needClientAuth, Boolean wantClientAuth, Integer headerBufferSize) {
-        SslContextFactory factory = new SslContextFactory();
-        factory.setExcludeCipherSuites("SSL_RSA_WITH_RC4_128_MD5", "SSL_RSA_WITH_RC4_128_SHA");
-        factory.setExcludeProtocols("SSLv3");
-        factory.setRenegotiationAllowed(false);
-        factory.setKeyStorePath(ksPath);
-        factory.setKeyStoreType(ksType);
-        factory.setKeyStorePassword(ksPassword);
-        factory.setKeyManagerPassword(keyPassword);
-
-        if (tsPath != null && tsPassword != null && tsType != null) {
-            factory.setTrustStorePath(tsPath);
-            factory.setTrustStoreType(tsType);
-            factory.setTrustStorePassword(tsPassword);
-        }
-
-        if (needClientAuth != null && needClientAuth) {
-            factory.setNeedClientAuth(true);
-        } else if (wantClientAuth != null && wantClientAuth) {
-            factory.setWantClientAuth(true);
-        }
-
-        HttpConfiguration httpsConfig = new HttpConfiguration();
-        httpsConfig.addCustomizer(new SecureRequestCustomizer());
-        if (null != headerBufferSize) {
-            httpsConfig.setRequestHeaderSize(headerBufferSize);
-        }
-        ServerConnector sslConnector = new ServerConnector(server,
-                                                           new SslConnectionFactory(factory, HttpVersion.HTTP_1_1.asString()),
-                                                           new HttpConnectionFactory(httpsConfig));
-        sslConnector.setPort(port);
-        return sslConnector;
-    }
-
-    public static void configSsl(Server server, Integer port, String ksPath, String ksPassword, String ksType,
-                                 String keyPassword, String tsPath, String tsPassword, String tsType,
-                                 Boolean needClientAuth, Boolean wantClientAuth) {
-        configSsl(server, port, ksPath, ksPassword, ksType, keyPassword,
-                  tsPath, tsPassword, tsType, needClientAuth, wantClientAuth, null);
-    }
-
-    public static void configSsl(Server server, Integer port, String ksPath, String ksPassword, String ksType,
-                                 String keyPassword, String tsPath, String tsPassword, String tsType,
-                                 Boolean needClientAuth, Boolean wantClientAuth, Integer headerBufferSize) {
-        if (port > 0) {
-            server.addConnector(mkSslConnector(server, port, ksPath, ksPassword, ksType, keyPassword,
-                                               tsPath, tsPassword, tsType, needClientAuth, wantClientAuth, headerBufferSize));
-        }
-    }
-
-    public static FilterHolder corsFilterHandle() {
-        FilterHolder filterHolder = new FilterHolder(new CrossOriginFilter());
-        filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_ORIGINS_PARAM, "*");
-        filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_ORIGINS_PARAM, "GET, POST, PUT");
-        filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_ORIGINS_PARAM,
-                                      "X-Requested-With, X-Requested-By, Access-Control-Allow-Origin, Content-Type, Content-Length, " +
-                                      "Accept, Origin");
-        filterHolder.setInitParameter(CrossOriginFilter.ACCESS_CONTROL_ALLOW_ORIGIN_HEADER, "*");
-        return filterHolder;
-    }
-
-    public static FilterHolder mkAccessLoggingFilterHandle() {
-        return new FilterHolder(new AccessLoggingFilter());
-    }
-
-    public static void configFilter(Server server, Servlet servlet, List<FilterConfiguration> filtersConfs) {
-        configFilter(server, servlet, filtersConfs, null);
-    }
-
-    public static void configFilter(Server server, Servlet servlet, List<FilterConfiguration> filtersConfs, Map<String, String> params) {
-        if (filtersConfs != null) {
-            ServletHolder servletHolder = new ServletHolder(servlet);
-            servletHolder.setInitOrder(0);
-            if (params != null) {
-                servletHolder.setInitParameters(params);
-            }
-            ServletContextHandler context = new ServletContextHandler(server, "/");
-            context.addServlet(servletHolder, "/");
-            configFilters(context, filtersConfs);
-            server.setHandler(context);
-        }
-    }
-
-    public static void configFilters(ServletContextHandler context, List<FilterConfiguration> filtersConfs) {
-        context.addFilter(corsFilterHandle(), "/*", EnumSet.allOf(DispatcherType.class));
-        for (FilterConfiguration filterConf : filtersConfs) {
-            String filterName = filterConf.getFilterName();
-            String filterClass = filterConf.getFilterClass();
-            Map<String, String> filterParams = filterConf.getFilterParams();
-            if (filterClass != null) {
-                FilterHolder filterHolder = new FilterHolder();
-                filterHolder.setClassName(filterClass);
-                if (filterName != null) {
-                    filterHolder.setName(filterName);
-                } else {
-                    filterHolder.setName(filterClass);
-                }
-                if (filterParams != null) {
-                    filterHolder.setInitParameters(filterParams);
-                } else {
-                    filterHolder.setInitParameters(new HashMap<>());
-                }
-                context.addFilter(filterHolder, "/*", EnumSet.allOf(DispatcherType.class));
-            }
-        }
-        context.addFilter(mkAccessLoggingFilterHandle(), "/*", EnumSet.allOf(DispatcherType.class));
-    }
-
-    /**
-     * Construct a Jetty Server instance.
-     */
-    public static Server jettyCreateServer(Integer port, String host, Integer httpsPort) {
-        return jettyCreateServer(port, host, httpsPort, null);
-    }
-
-    /**
-     * Construct a Jetty Server instance.
-     */
-    public static Server jettyCreateServer(Integer port, String host, Integer httpsPort, Integer headerBufferSize) {
-        Server server = new Server();
-
-        if (httpsPort == null || httpsPort <= 0) {
-            HttpConfiguration httpConfig = new HttpConfiguration();
-            httpConfig.setSendDateHeader(true);
-            if (null != headerBufferSize) {
-                httpConfig.setRequestHeaderSize(headerBufferSize);
-            }
-            ServerConnector httpConnector = new ServerConnector(server, new HttpConnectionFactory(httpConfig));
-            httpConnector.setPort(ObjectReader.getInt(port, 80));
-            httpConnector.setIdleTimeout(200000);
-            httpConnector.setHost(host);
-            server.addConnector(httpConnector);
-        }
-
-        return server;
-    }
-
-    /**
-     * Modified version of run-jetty
-     * Assumes configurator sets handler.
-     */
-    public static void stormRunJetty(Integer port, String host, Integer httpsPort, Integer headerBufferSize,
-                                     IConfigurator configurator) throws Exception {
-        Server s = jettyCreateServer(port, host, httpsPort, headerBufferSize);
-        if (configurator != null) {
-            configurator.execute(s);
-        }
-        s.start();
-    }
-
-    public static void stormRunJetty(Integer port, Integer headerBufferSize, IConfigurator configurator) throws Exception {
-        stormRunJetty(port, null, null, headerBufferSize, configurator);
-    }
-
-    public static String wrapJsonInCallback(String callback, String response) {
-        return callback + "(" + response + ");";
-    }
-
-    public static Map getJsonResponseHeaders(String callback, Map headers) {
-        Map<String, String> headersResult = new HashMap<>();
-        headersResult.put("Cache-Control", "no-cache, no-store");
-        headersResult.put("Access-Control-Allow-Origin", "*");
-        headersResult.put("Access-Control-Allow-Headers",
-                          "Content-Type, Access-Control-Allow-Headers, Access-Controler-Allow-Origin, X-Requested-By, X-Csrf-Token, " +
-                          "Authorization, X-Requested-With");
-        if (callback != null) {
-            headersResult.put("Content-Type", "application/javascript;charset=utf-8");
-        } else {
-            headersResult.put("Content-Type", "application/json;charset=utf-8");
-        }
-        if (headers != null) {
-            headersResult.putAll(headers);
-        }
-        return headersResult;
-    }
-
-    public static String getJsonResponseBody(Object data, String callback, boolean needSerialize) {
-        String serializedData = needSerialize ? JSONValue.toJSONString(data) : (String) data;
-        return callback != null ? wrapJsonInCallback(callback, serializedData) : serializedData;
-    }
-
-    public static Map exceptionToJson(Exception ex, int statusCode) {
-        StringWriter sw = new StringWriter();
-        ex.printStackTrace(new PrintWriter(sw));
-        return ImmutableMap.of("error", statusCode + " " + HttpStatus.getMessage(statusCode), "errorMessage", sw.toString());
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/034ac677/storm-core/src/jvm/org/apache/storm/utils/Monitor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/Monitor.java b/storm-core/src/jvm/org/apache/storm/utils/Monitor.java
index 8e3ce27..9ff2d36 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/Monitor.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/Monitor.java
@@ -1,7 +1,7 @@
 /**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  The ASF licenses this file to you under the Apache License, Version
- * 2.0 (the "License"); you may not use this file except in compliance with the License.  You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership. The ASF licenses this file to you under the Apache License, Version
+ * 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at
  *
  * http://www.apache.org/licenses/LICENSE-2.0
  *

http://git-wip-us.apache.org/repos/asf/storm/blob/034ac677/storm-core/src/ui/public/component.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html
deleted file mode 100644
index cee4ce4..0000000
--- a/storm-core/src/ui/public/component.html
+++ /dev/null
@@ -1,569 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01//EN" "http://www.w3.org/TR/html4/strict.dtd">
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<html><head>
-<meta charset="UTF-8">
-<meta name="viewport" content="width=device-width, initial-scale=1">
-<title>Storm UI</title>
-<link href="/css/bootstrap-3.3.1.min.css" rel="stylesheet" type="text/css">
-<link href="/css/jquery.dataTables.1.10.4.min.css" rel="stylesheet" type="text/css">
-<link href="/css/dataTables.bootstrap.css" rel="stylesheet" type="text/css">
-<link href="/css/style.css?_ts=${packageTimestamp}" rel="stylesheet" type="text/css">
-<script src="/js/jquery-1.11.1.min.js" type="text/javascript"></script>
-<script src="/js/jquery.dataTables.1.10.4.min.js" type="text/javascript"></script>
-<script src="/js/jquery.cookies.2.2.0.min.js" type="text/javascript"></script>
-<script src="/js/jquery.mustache.js" type="text/javascript"></script>
-<script src="/js/url.min.js" type="text/javascript"></script>
-<script src="/js/bootstrap-3.3.1.min.js" type="text/javascript"></script>
-<script src="/js/jquery.blockUI.min.js" type="text/javascript"></script>
-<script src="/js/moment.min.js" type="text/javascript"></script>
-<script src="/js/dataTables.bootstrap.min.js" type="text/javascript"></script>
-<script src="/js/script.js?_ts=${packageTimestamp}" type="text/javascript"></script>
-</head>
-<body>
-<div class="container-fluid">
-  <div class="row">
-    <div class="col-md-11">
-      <h1><a href="/">Storm UI</a></h1>
-    </div>
-    <div id="ui-user" class="col-md-1"></div>
-  </div>
-  <div class="row">
-    <div id="component-summary" class="col-md-12"></div>
-  </div>
-  <div class="row">
-    <div id="component-actions" class="col-md-12"></div>
-  </div>
-  <div class="row">
-    <div id="component-stats-detail" class="col-md-12"></div>
-  </div>
-  <div class="row">
-    <div id="component-input-stats" class="col-md-12"></div>
-  </div>
-  <div class="row">
-    <div id="component-output-stats" class="col-md-12"></div>
-  </div>
-  <div class="row">
-    <div id="profiler-control" class="col-md-12"></div>
-  </div>
-  <div class="row">
-    <div id="component-executor-stats" class="col-md-12"></div>
-  </div>
-  <div class="row">
-    <div id="component-errors" class="col-md-12"></div>
-  </div>
-  <div class="row">
-    <div id="json-response-error" class="col-md-12"></div>
-  </div>
-  <div class="row">
-    <div class="col-md-1">
-      <p id="toggle-switch" style="display: block;" class="js-only"></p>
-    </div>
-  </div>
-  <div>
-    <p id="page-rendered-at-timestamp"></p>
-  </div>
-<script>
-$(document).ajaxStop($.unblockUI);
-$(document).ajaxStart(function(){
-    $.blockUI({ message: '<img src="images/spinner.gif" /> <h3>Loading component summary...</h3>'});
-});
-function jsError(other) {
-    try {
-      other();
-    } catch (err) {
-      getStatic("/templates/json-error-template.html", function(template) {
-        $("#json-response-error").append(Mustache.render($(template).filter("#json-error-template").html(),{error: "JS Error", errorMessage: err}));
-      });
-    }
-}
-
-function setWorkerActionCheckboxesClickCallback() {
-    $('#executor-stats-table tbody tr td')
-        .on('click', ".workerActionCheckbox", workerActionSelectedClicked);
-}
-
-function redrawExecutorTable() {
-    var table = $('#executor-stats-table').DataTable();
-    var data = table.data();
-    // Datatables will not render for display when draw() is called.
-    // so we must clear the data and add it back.
-    table.clear().rows.add(data).draw(false /* keep current page */);
-}
-
-function disableWorkerActionButtons(disable) {
-    $('span#workerActionButtons > input[type="button"]').each(function(e) {
-      this.disabled = disable;
-    });
-}
-
-function setWorkerActionSelected(key, isSelected) {
-    if (isSelected) {
-        workerActionSelected[key] = true;
-        disableWorkerActionButtons(false);
-    } else {
-        delete workerActionSelected[key];
-        if (Object.keys(workerActionSelected) == 0) {
-          disableWorkerActionButtons(true);
-        }
-    }
-    redrawExecutorTable();
-}
-
-function workerActionSelectedClicked() {
-    if (this.checked) {
-        setWorkerActionSelected(this.value, true);
-    } else {
-        setWorkerActionSelected(this.value, false);
-    }
-};
-
-workerActionSelected = {};
-
-$(document).ready(function() {
-    var componentId = $.url("?id");
-    var topologyId = $.url("?topology_id");
-    var tableStateKey = ":".concat(topologyId, ":", componentId);
-    var window = $.url("?window");
-    var sys = $.cookies.get("sys") || "false";
-    var url = "/api/v1/topology/"+topologyId+"/component/"+componentId+"?sys="+sys;
-    if(window) url += "&window="+window;
-
-    $.getJSON("/api/v1/cluster/configuration",function(response,status,jqXHR) {
-        $.extend( $.fn.dataTable.defaults, {
-          stateSave: true,
-          stateSaveCallback: function (oSettings, oData) {
-            sessionStorage.setItem( oSettings.sTableId.concat(tableStateKey), JSON.stringify(oData) );
-          },
-          stateLoadCallback: function (oSettings) {
-            return JSON.parse( sessionStorage.getItem(oSettings.sTableId.concat(tableStateKey)) );
-          },
-          lengthMenu: [[20,40,60,100,-1], [20, 40, 60, 100, "All"]],
-          pageLength: response["ui.pagination"]
-        });
-    });
-
-    renderToggleSys($("#toggle-switch"));
-    $.ajaxSetup({
-        "error":function(jqXHR,textStatus,response) {
-            var errorJson = jQuery.parseJSON(jqXHR.responseText);
-            getStatic("/templates/json-error-template.html", function(template) {
-                $("#json-response-error").append(Mustache.render($(template).filter("#json-error-template").html(),errorJson));
-            });
-        }
-    });
-
-    function renderSupervisorPageLink(data, type, row, meta) {
-        return type === 'display' ? 
-                   ("<a href='/supervisor.html?host=" + data + "'>" + data + "</a>") :
-                   data; 
-    }
-
-    function renderActionCheckbox(data, type, row, meta) {
-      var host_port = row[2]+':'+$(row[3])[0].text;
-      switch(type) {
-        case 'filter':
-        case 'display':
-        case 'type':
-          var checkedString =
-              host_port in workerActionSelected ? ' checked' : '';
-          var checkboxId ='workerActionCheckbox_'+row[0].replace(/\[(.*)\]/,"$1");
-          // To present a link to debugging output without needing to change
-          // the UI REST API, we must parse the logviewer URL.
-          var loc = $(row[3])[0]; // logviewer URL
-          return '<input type="checkbox" class="workerActionCheckbox"'+
-              'id="'+checkboxId+'" value="'+host_port+'"'+checkedString+'/> '+
-              '<a href="'+loc.protocol+'//'+loc.host+'/api/v1/dumps/'+topologyId+'/'+
-              encodeURIComponent(host_port)+'">files</a>';
-          break;
-        case 'sort':
-        default:
-          return host_port in workerActionSelected ? 1 : 0;
-      }
-    }
-
-    $.getJSON(url,function(response,status,jqXHR) {
-        var uiUser = $("#ui-user");
-        getStatic("/templates/user-template.html", function(template) {
-            uiUser.append(Mustache.render($(template).filter("#user-template").html(),response));
-            $('#ui-user [data-toggle="tooltip"]').tooltip()
-        });
-
-        var topologyUrl = "/api/v1/topology/"+topologyId;
-
-        var eventLoggers = (function() {
-              $.ajaxSetup({
-                  async: false
-              });
-              var eventLoggers;
-              $.getJSON(topologyUrl, function(response, status, jqXHR) {
-                  eventLoggers = response["configuration"]["topology.eventlogger.executors"];
-               });
-               $.ajaxSetup({
-                  async: true
-              })
-              return eventLoggers;
-         })();
-
-        var componentSummary = $("#component-summary");
-        var componentActions = $("#component-actions");
-        var buttonJsonData = componentActionJson(response["encodedTopologyId"], response["encodedId"], response["id"],
-                                                 response["topologyStatus"], eventLoggers, response["debug"], response["samplingPct"]);
-        var componentStatsDetail = $("#component-stats-detail")
-        var inputStats = $("#component-input-stats");
-        var outputStats = $("#component-output-stats");
-        var profilerControl = $("#profiler-control");
-        var executorStats = $("#component-executor-stats");
-        var componentErrors = $("#component-errors");
-        getStatic("/templates/component-page-template.html", function(template) {
-            response["profilerActive"] = $.map(response["profilerActive"], function(active_map) {
-                var date = new Date();
-                var millis = date.getTime() + parseInt(active_map["timestamp"]);
-                date = new Date(millis);
-                active_map["timestamp"] = date.toTimeString();
-                return active_map;
-            });
-
-            jsError(function() {
-              componentSummary.append(Mustache.render($(template).filter("#component-summary-template").html(),response));
-            });
-
-            jsError(function() {
-              componentActions.append(Mustache.render($(template).filter("#component-actions-template").html(),buttonJsonData));
-            });
-
-            if (response["profilingAndDebuggingCapable"] == true) {
-                jsError(function () {
-                    var part = $(template).filter('#profiler-active-partial').html();
-                    var partials = {"profilerActive": part};
-                    profilerControl.append(Mustache.render($(template).filter("#profiling-template").html(), response, partials));
-                });
-            }
-
-            if(response["componentType"] == "spout") {
-                componentStatsDetail.append(Mustache.render($(template).filter("#spout-stats-detail-template").html(),response));
-                //window, emitted, transferred, complete latency, acked, failed
-                $("#spout-stats-table").DataTable({
-                  paging: false,
-                  info: false,
-                  searching: false,
-                  columnDefs: [
-                    {type: "num", targets: [1, 2, 3, 4, 5]},
-                    {type: "time-str", targets: [0]}
-                  ]
-                });
-
-                outputStats.append(Mustache.render($(template).filter("#output-stats-template").html(),response));
-                //stream, emitted, transferred, compltete latency, acked, failed
-                dtAutoPage("#output-stats-table", {
-                  columnDefs: [
-                    {type: "num", targets: [1, 2, 3, 4, 5]}
-                  ]
-                });
-
-                executorStats.append(Mustache.render($(template).filter("#executor-stats-template").html(),response));
-                //id, uptime, host, port, actions, emitted, transferred, complete latency, acked, failed
-                dtAutoPage("#executor-stats-table", {
-                  columnDefs: [
-                    {render: renderSupervisorPageLink, searchable: true, targets: [2]},
-                    {render: renderActionCheckbox, searchable: false, targets: [4]},
-                    {type: "num", targets: [5, 6, 7, 8, 9]},
-                    {type: "time-str", targets: [1]},
-                  ]
-                }).on("draw", function(e,s) {setWorkerActionCheckboxesClickCallback()});
-            } else {
-                componentStatsDetail.append(Mustache.render($(template).filter("#bolt-stats-template").html(),response));
-                //window, emitted, transferred, execute latency, executed, process latency, acked, failed
-                dtAutoPage("#bolt-stats-table", {
-                  columnDefs: [
-                    {type: "num", targets: [1, 2, 3, 4, 5, 6, 7]},
-                    {type: "time-str", targets: [0]}
-                  ]
-                });
-
-                inputStats.append(Mustache.render($(template).filter("#bolt-input-stats-template").html(),response));
-                //component, stream, execute latency, executed, process latency, acked, failed
-                dtAutoPage("#bolt-input-stats-table", {
-                  columnDefs: [
-                    {type: "num", targets: [2, 3, 4, 5, 6]}
-                  ]
-                });
-
-                outputStats.append(Mustache.render($(template).filter("#bolt-output-stats-template").html(),response));
-                //stream, emitted, transferred
-                dtAutoPage("#bolt-output-stats-table", {
-                  columnDefs: [
-                    {type: "num", targets: [1, 2]}
-                  ]
-                });
-
-                executorStats.append(Mustache.render($(template).filter("#bolt-executor-template").html(),response));
-                //id, uptime, host, port, actions, emitted, transferred, capacity, execute latency, executed, process latency, acked, failed
-                dtAutoPage("#executor-stats-table", {
-                  columnDefs: [
-                    {render: renderSupervisorPageLink, searchable: true, targets: [2]},
-                    {render: renderActionCheckbox, searchable: false, targets: [4]},
-                    {type: "num", targets: [5, 6, 7, 8, 9, 10, 11, 12]},
-                    {type: "time-str", targets: [1]},
-                  ]
-                }).on("draw", function(e,s) {setWorkerActionCheckboxesClickCallback()});
-            }
-            setWorkerActionCheckboxesClickCallback();
-            componentErrors.append(Mustache.render($(template).filter("#component-errors-template").html(),formatErrorTimeSecs(response)));
-            //time, error
-            dtAutoPage("#component-errors-table", {});
-
-            var errorTimeCells = document.getElementsByClassName("errorTimeSpan");
-            for (i = 0; i < errorTimeCells.length; i++)
-            {
-              var timeInMilliseconds = errorTimeCells[i].id * 1000;
-              var time = parseInt(timeInMilliseconds);
-              var date = new Date(time);
-              errorTimeCells[i].innerHTML = date.toJSON();
-            }
-
-            var errorCells = document.getElementsByClassName("errorSpan");
-            for (i =0; i < errorCells.length; i++)
-            {
-              var timeLapsedInSecs = errorCells[i].id;
-              if (parseInt(timeLapsedInSecs) < 1800) {
-                errorCells[i].style.color = "#9d261d";
-                errorCells[i].style.borderBottomColor = "#9d261d";
-              }
-              errorCells[i].style.whiteSpace = "pre";
-            }
-            $('#component-summary [data-toggle="tooltip"]').tooltip();
-            $('#component-actions [data-toggle="tooltip"]').tooltip();
-            $('#component-stats-detail [data-toggle="tooltip"]').tooltip();
-            $('#component-input-stats [data-toggle="tooltip"]').tooltip();
-            $('#component-output-stats [data-toggle="tooltip"]').tooltip();
-            $('#component-executor-stats [data-toggle="tooltip"]').tooltip();
-            $('#component-errors [data-toggle="tooltip"]').tooltip();
-        });
-    });
-});
-
-getPageRenderedTimestamp("page-rendered-at-timestamp");
-
-function start_profiling() {
-    if (!confirmAction("start profiling")) return false;
-    var topologyId = $.url("?topology_id");
-    var timeout = $("#timeout").val();
-
-    if(timeout == "") { timeout = 10; }
-    if(isNaN(parseFloat(timeout)) || !isFinite(timeout)) {
-        alert("Must specify a numeric timeout");
-        return;
-    }
-
-    var failed = {}
-    var passed = {}
-    Object.keys(workerActionSelected).forEach(function (id) {
-        var url = "/api/v1/topology/"+topologyId+"/profiling/start/" + id + "/" + timeout;
-        $.get(url, function(response,status,jqXHR) {
-            jsError(function() {
-                getStatic("/templates/component-page-template.html", function(template) {
-                    var host_port_split = id.split(":");
-                    var host = host_port_split[0];
-                    var port = host_port_split[1];
-                    var millis = new Date().getTime() + (timeout * 60000);
-                    var timestamp = new Date(millis).toTimeString();
-
-                    var mustache = Mustache.render($(template).filter("#profiler-active-partial").html(), {"profilerActive": [{
-                        "host": host,
-                        "port": port,
-                        "timestamp": timestamp,
-                        "dumplink": response["dumplink"]}]});
-                    $("#profiler-table-body").append(mustache);
-                });
-            });
-        })
-        .fail(function(response) {
-            failed[id] = response;
-        });
-        if (!(id in failed)) {
-            passed[id] = true;
-            setWorkerActionSelected(id, false);
-        }
-    });
-    if (Object.keys(failed) > 0) {
-        alert('Failed to start profiling for '+JSON.stringify(Object.keys(failed)));
-    } else {
-        alert('Sent requests to start profiling for '+JSON.stringify(Object.keys(passed)));
-    }
-    redrawExecutorTable();
-}
-
-function stop_profiling(id) {
-    if (!confirmAction("stop profiling")) return false;
-    var topologyId = $.url("?topology_id");
-    var url = "/api/v1/topology/"+topologyId+"/profiling/stop/" + id;
-
-    $("#stop_" + id).prop('disabled', true);
-    setTimeout(function(){ $("#stop_" + id).prop('disabled', false); }, 5000);
-    
-    $.get(url, function(response,status,jqXHR) {
-        alert("Submitted request to stop profiling...");
-    })
-    .fail(function(response) {
-        alert( "Stopping profiler for " + id + " failed: \n" + JSON.stringify(response));
-    });
-    
-}
-
-function dump_profile(id) {
-    if (!confirmAction("dump profile")) return false;
-    var topologyId = $.url("?topology_id");
-    var url = "/api/v1/topology/"+topologyId+"/profiling/dumpprofile/" + id;
-
-    $("#dump_profile_" + id).prop('disabled', true);
-    setTimeout(function(){ $("#dump_profile_" + id).prop('disabled', false); }, 5000);
-    
-    $.get(url, function(response,status,jqXHR) {
-        alert("Submitted request to dump profile snapshot...");
-    })
-    .fail(function(response) {
-        alert( "Dumping profile data for " + id + " failed: \n" + JSON.stringify(response));
-    });
-}
-
-// Create jstack output for all selected workers.
-function dump_jstacks() {
-    if (!confirmAction("dump jstack")) return false;
-    var topologyId = $.url("?topology_id");
-    var failed = {}
-    var passed = {}
-    Object.keys(workerActionSelected).forEach(function (id) {
-        var url = "/api/v1/topology/"+topologyId+"/profiling/dumpjstack/" + id;
-
-        $("#dump_jstack_" + id).prop('disabled', true);
-        setTimeout(function(){ $("#dump_jstack_" + id).prop('disabled', false); }, 5000);
-
-        $.get(url).fail(function(response) {
-            failed[id] = response;
-        });
-        if (!(id in failed)) {
-            passed[id] = true;
-            setWorkerActionSelected(id, false);
-        }
-    });
-    if (Object.keys(failed) > 0) {
-        alert('Failed to create jstack output for '+JSON.stringify(Object.keys(failed)));
-    } else {
-        alert('Sent requests to create jstack output for '+JSON.stringify(Object.keys(passed)));
-    }
-    redrawExecutorTable();
-}
-
-// Create jstack output for the worker with the given id.
-function dump_jstack(id) {
-    if (!confirmAction("dump jstack")) return false;
-    var topologyId = $.url("?topology_id");
-    var url = "/api/v1/topology/"+topologyId+"/profiling/dumpjstack/" + id;
-
-    $("#dump_jstack_" + id).prop('disabled', true);
-    setTimeout(function(){ $("#dump_jstack_" + id).prop('disabled', false); }, 5000);
-    
-    $.get(url, function(response,status,jqXHR) {
-        alert("Submitted request for jstack dump...");
-    })
-    .fail(function(response) {
-        alert( "Dumping JStack for " + id + " failed: \n" + JSON.stringify(response));
-    });
-}
-
-function restart_worker_jvms() {
-    if (!confirmAction("restart worker")) return false;
-    var topologyId = $.url("?topology_id");
-    var failed = {}
-    var passed = {}
-    Object.keys(workerActionSelected).forEach(function (id) {
-        var url = "/api/v1/topology/"+topologyId+"/profiling/restartworker/" + id;
-
-        $("#restart_worker_jvm_" + id).prop('disabled', true);
-        setTimeout(function(){ $("#restart_worker_jvm_" + id).prop('disabled', false); }, 5000);
-
-        $.get(url).fail(function(response) {
-            failed[id] = response;
-        });
-        if (!(id in failed)) {
-            passed[id] = true;
-            setWorkerActionSelected(id, false);
-        }
-    });
-    if (Object.keys(failed) > 0) {
-        alert('Failed to restart for '+JSON.stringify(Object.keys(failed)));
-    } else {
-        alert('Sent requests to restart '+JSON.stringify(Object.keys(passed)));
-    }
-    redrawExecutorTable();
-}
-
-// Create java heap output for all selected workers.
-function dump_heaps() {
-    if (!confirmAction("dump heap")) return false;
-    var topologyId = $.url("?topology_id");
-    var failed = {}
-    var passed = {}
-    Object.keys(workerActionSelected).forEach(function (id) {
-        var url = "/api/v1/topology/"+topologyId+"/profiling/dumpheap/" + id;
-        var heap = $("#dump_heap_" + id);
-        $("#dump_heap_" + id).prop('disabled', true);
-        setTimeout(function(){ $("#dump_heap_" + id).prop('disabled', false); }, 5000);
-
-        $.get(url).fail(function(response) {
-            failed[id] = response;
-        });
-        if (!(id in failed)) {
-            passed[id] = true;
-            setWorkerActionSelected(id, false);
-        }
-    });
-    if (Object.keys(failed) > 0) {
-        alert('Failed to create Java heap output for '+JSON.stringify(Object.keys(failed)));
-    } else {
-        alert('Sent requests to create Java heap output for '+JSON.stringify(Object.keys(passed)));
-    }
-    redrawExecutorTable();
-}
-
-// Create java heap output for the worker with the given id.
-function dump_heap(id) {
-    if (!confirmAction("dump heap")) return false;
-    var topologyId = $.url("?topology_id");
-    var url = "/api/v1/topology/"+topologyId+"/profiling/dumpheap/" + id;
-    var heap = $("#dump_heap_" + id);
-    $("#dump_heap_" + id).prop('disabled', true);
-    setTimeout(function(){ $("#dump_heap_" + id).prop('disabled', false); }, 5000);
-    
-    $.get(url, function(response,status,jqXHR) {
-        alert("Submitted request for jmap dump...");
-    })
-    .fail(function(response) {
-        alert( "Dumping Heap for " + id + " failed: \n" + JSON.stringify(response));
-    });
-}
-
-// Confirm an action
-function confirmAction(actionText){
-    return confirm('Do you really want to ' + actionText + '?');
-}
-
-</script>
-</div>
-</body>
-</html>