[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-08-03 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/2752


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-08-03 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r207554122
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIHelpers.java ---
@@ -0,0 +1,1941 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF)
+ * under one or more contributor license agreements.
+ * See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version
+ * 2.0 (the "License"); you may not use this file except in compliance 
with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an "AS IS" 
BASIS,
+ * WITHOUT 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.daemon.ui;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.net.URLEncoder;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import javax.servlet.DispatcherType;
+import javax.servlet.Servlet;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import org.apache.storm.Config;
+import org.apache.storm.Constants;
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.BoltAggregateStats;
+import org.apache.storm.generated.ClusterSummary;
+import org.apache.storm.generated.CommonAggregateStats;
+import org.apache.storm.generated.ComponentAggregateStats;
+import org.apache.storm.generated.ComponentPageInfo;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorSummary;
+import org.apache.storm.generated.GetInfoOptions;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.Grouping;
+import org.apache.storm.generated.KillOptions;
+import org.apache.storm.generated.LogConfig;
+import org.apache.storm.generated.LogLevel;
+import org.apache.storm.generated.LogLevelAction;
+import org.apache.storm.generated.Nimbus;
+import org.apache.storm.generated.NimbusSummary;
+import org.apache.storm.generated.NodeInfo;
+import org.apache.storm.generated.NumErrorsChoice;
+import org.apache.storm.generated.OwnerResourceSummary;
+import org.apache.storm.generated.ProfileAction;
+import org.apache.storm.generated.ProfileRequest;
+import org.apache.storm.generated.RebalanceOptions;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.SupervisorPageInfo;
+import org.apache.storm.generated.SupervisorSummary;
+import org.apache.storm.generated.TopologyHistoryInfo;
+import org.apache.storm.generated.TopologyInfo;
+import org.apache.storm.generated.TopologyPageInfo;
+import org.apache.storm.generated.TopologyStats;
+import org.apache.storm.generated.TopologySummary;
+import org.apache.storm.generated.WorkerSummary;
+import org.apache.storm.logging.filters.AccessLoggingFilter;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.TopologySpoutLag;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.utils.VersionInfo;
+import org.apache.storm.utils.WebAppUtils;
+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;

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-08-03 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r207551893
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/AuthorizedUserFilter.java
 ---
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.daemon.ui.filters;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.Map;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.ui.resources.NimbusOp;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+try {
+uiImpersonationHandler = StormCommon.mkAuthorizationHandler(
+(String) 
conf.get(DaemonConfig.NIMBUS_IMPERSONATION_AUTHORIZER), conf);
+uiAclHandler = StormCommon.mkAuthorizationHandler(
+(String) conf.get(DaemonConfig.NIMBUS_AUTHORIZER), 
conf);
+} catch (IllegalAccessException | InstantiationException | 
ClassNotFoundException e) {
+LOG.error("Error initializing AuthorizedUserFilter: ", e);
+throw new RuntimeException(e);
+}
+}
+
+@Override
+public void filter(ContainerRequestContext containerRequestContext) 
throws IOException {
+NimbusOp annotation = 
resourceInfo.getResourceMethod().getAnnotation(NimbusOp.class);
+if (annotation == null) {
+return;
+}
+String op = annotation.value();
+if (op == null) {
+return;
+}
+
+Map topoConf = null;
+if 
(containerRequestContext.getUriInfo().getPathParameters().containsKey("id")) {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+try {
+topoConf = (Map) 
JSONValue.parse(nimbusClient.getClient().getTopologyConf(
+
containerRequestContext.getUriInfo().getPathParameters().get("id").get(0)));
+} catch (TException e) {
+e.printStackTrace();
--- End diff --

This appears to be the part that is causing the issues.  Printing the 
stacktrace is not logging it, so it will not show up where we want it.  And if 
we are handling it properly it will not matter.  An IOException is not going to 
be mapper properly into the result we want.  Also all errors we don't expect 
500 errors will be returned as HTML.  Lets have a generic mapper that can take 
any Throwable and turn it into a JSON response?  Also at the same time can we 
change this IOException so it has e as the cause, and in the Generic exception 
mapper can we have it look for a cause of Authroized, etc and map it to the 
proper response?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-08-03 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r207542398
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/webapp/LogviewerResource.java
 ---
@@ -137,7 +137,7 @@ public Response searchLogs(@Context HttpServletRequest 
request) throws IOExcepti
 String user = httpCredsHandler.getUserName(request);
 String topologyId = request.getParameter("topoId");
 String portStr = request.getParameter("port");
-String callback = request.getParameter("callback");
+String callback = request.getParameter("callbackParameterName");
--- End diff --

So why are we changing `/searchLogs?callback=bar` to 
`/searchLogs?callbackParameterName=bar`?  If that is what we are doing it is a 
regression, if not I am really missing something.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-30 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r206240455
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/resources/StormApiResource.java
 ---
@@ -0,0 +1,702 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.daemon.ui.resources;
+
+import com.codahale.metrics.Meter;
+import java.net.URLDecoder;
+import java.util.Map;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import org.apache.storm.daemon.ui.UIHelpers;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private HttpServletRequest servletRequest;
+
+public static Map config = Utils.readStormConfig();
+
+public static Meter clusterConfigurationRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-cluster-configuration-http-requests");
+
+public static Meter clusterSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-cluster-summary-http-requests");
+
+public static Meter nimbusSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-nimbus-summary-http-requests");
+
+public static Meter supervisorRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-supervisor-http-requests");
+
+public static Meter supervisorSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-supervisor-summary-http-requests");
+
+public static Meter allTopologiesSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-all-topologies-summary-http-requests");
+
+public static Meter topologyPageRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-topology-page-http-requests");
+
+public static Meter topologyMetricRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-topology-metric-http-requests");
+
+public static Meter buildVisualizationRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-build-visualization-http-requests");
+
+public static Meter mkVisualizationDataRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-mk-visualization-data-http-requests");
+
+public static Meter componentPageRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-component-page-http-requests");
+
+public static Meter logConfigRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-log-config-http-requests");
+
+public static Meter activateTopologyRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-activate-topology-http-requests");
+
+public static Meter deactivateTopologyRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-deactivate-topology-http-requests");
+
+public static Meter debugTopologyRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-debug-topology-http-requests");
+
+public static Meter componentOpResponseRequestMeter =
+

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-30 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r206238030
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/AuthorizedUserFilter.java
 ---
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.daemon.ui.filters;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.Map;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.ui.resources.NimbusOp;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+try {
+uiImpersonationHandler = StormCommon.mkAuthorizationHandler(
+(String) 
conf.get(DaemonConfig.NIMBUS_IMPERSONATION_AUTHORIZER), conf);
+uiAclHandler = StormCommon.mkAuthorizationHandler(
+(String) conf.get(DaemonConfig.NIMBUS_AUTHORIZER), 
conf);
+} catch (IllegalAccessException | InstantiationException | 
ClassNotFoundException e) {
+LOG.error("Error initializing AuthorizedUserFilter: ", e);
+throw new RuntimeException(e);
+}
+}
+
+@Override
+public void filter(ContainerRequestContext containerRequestContext) 
throws IOException {
+NimbusOp annotation = 
resourceInfo.getResourceMethod().getAnnotation(NimbusOp.class);
+if (annotation == null) {
+return;
+}
+String op = annotation.value();
+if (op == null) {
+return;
+}
+
+Map topoConf = null;
+if 
(containerRequestContext.getUriInfo().getPathParameters().containsKey("id")) {
--- End diff --

It feels a bit like `?id=` is always the topology id is a bit of magic. 
Could we have a follow on JIRA to make it so each API can configure it, perhaps 
through the annotation?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-30 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r206238984
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/resources/NimbusOp.java 
---
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.daemon.ui.resources;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+@Documented
+@Retention(value = RetentionPolicy.RUNTIME)
+public @interface NimbusOp {
--- End diff --

nit: because this really is controlling security, could we rename it to 
something like `AuthNimbusOp`, or something that implies security a bit more?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-30 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r206235453
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/UIServer.java ---
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.daemon.ui;
+
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.EnumSet;
+import java.util.Map;
+import javax.servlet.DispatcherType;
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.drpc.webapp.ReqContextFilter;
+import 
org.apache.storm.daemon.ui.exceptionmappers.AuthorizationExceptionMapper;
+import org.apache.storm.daemon.ui.exceptionmappers.NotAliveExceptionMapper;
+import org.apache.storm.daemon.ui.exceptionmappers.TExceptionMapper;
+import org.apache.storm.daemon.ui.filters.AuthorizedUserFilter;
+import org.apache.storm.daemon.ui.filters.HeaderResponseFilter;
+import org.apache.storm.daemon.ui.filters.HeaderResponseServletFilter;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.security.auth.IHttpCredentialsPlugin;
+import org.apache.storm.security.auth.ServerAuthUtils;
+import org.apache.storm.utils.Utils;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.servlet.ServletContainer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+
+import static org.apache.storm.utils.ConfigUtils.FILE_SEPARATOR;
+import static org.apache.storm.utils.ConfigUtils.STORM_HOME;
+
+/**
+ * Main class.
+ *
+ */
+public class UIServer {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(UIServer.class);
+
+public static final String STORM_API_URL_PREFIX = "/api/v1/";
+
+/**
+ * addRequestContextFilter.
+ * @param context context
+ * @param configName configName
+ * @param conf conf
+ */
+public static void addRequestContextFilter(ServletContextHandler 
context,
+   String configName, 
Map conf) {
+IHttpCredentialsPlugin auth = 
ServerAuthUtils.getHttpCredentialsPlugin(conf, (String) conf.get(configName));
+ReqContextFilter filter = new ReqContextFilter(auth);
+context.addFilter(new FilterHolder(filter), "/*", 
EnumSet.allOf(DispatcherType.class));
+}
+
+/**
+ * main.
+ * @param args args
+ */
+public static void main(String[] args) {
+
+Server jettyServer = new Server();
+ServerConnector connector = new ServerConnector(jettyServer);
+// connector.setPort((Integer) conf.get(DaemonConfig.UI_PORT));
+connector.setPort(4443);
--- End diff --

Need to remove the hard coded port number


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-30 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r206239558
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/resources/StormApiResource.java
 ---
@@ -0,0 +1,702 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.daemon.ui.resources;
+
+import com.codahale.metrics.Meter;
+import java.net.URLDecoder;
+import java.util.Map;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import org.apache.storm.daemon.ui.UIHelpers;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private HttpServletRequest servletRequest;
+
+public static Map config = Utils.readStormConfig();
+
+public static Meter clusterConfigurationRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-cluster-configuration-http-requests");
+
+public static Meter clusterSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-cluster-summary-http-requests");
+
+public static Meter nimbusSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-nimbus-summary-http-requests");
+
+public static Meter supervisorRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-supervisor-http-requests");
+
+public static Meter supervisorSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-supervisor-summary-http-requests");
+
+public static Meter allTopologiesSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-all-topologies-summary-http-requests");
+
+public static Meter topologyPageRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-topology-page-http-requests");
+
+public static Meter topologyMetricRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-topology-metric-http-requests");
+
+public static Meter buildVisualizationRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-build-visualization-http-requests");
+
+public static Meter mkVisualizationDataRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-mk-visualization-data-http-requests");
+
+public static Meter componentPageRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-component-page-http-requests");
+
+public static Meter logConfigRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-log-config-http-requests");
+
+public static Meter activateTopologyRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-activate-topology-http-requests");
+
+public static Meter deactivateTopologyRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-deactivate-topology-http-requests");
+
+public static Meter debugTopologyRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-debug-topology-http-requests");
+
+public static Meter componentOpResponseRequestMeter =
+

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-30 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r206237782
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/filters/AuthorizedUserFilter.java
 ---
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.daemon.ui.filters;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.Map;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.daemon.ui.resources.NimbusOp;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+try {
+uiImpersonationHandler = StormCommon.mkAuthorizationHandler(
+(String) 
conf.get(DaemonConfig.NIMBUS_IMPERSONATION_AUTHORIZER), conf);
+uiAclHandler = StormCommon.mkAuthorizationHandler(
+(String) conf.get(DaemonConfig.NIMBUS_AUTHORIZER), 
conf);
+} catch (IllegalAccessException | InstantiationException | 
ClassNotFoundException e) {
+LOG.error("Error initializing AuthorizedUserFilter: ", e);
+throw new RuntimeException(e);
+}
+}
+
+@Override
+public void filter(ContainerRequestContext containerRequestContext) 
throws IOException {
+NimbusOp annotation = 
resourceInfo.getResourceMethod().getAnnotation(NimbusOp.class);
+if (annotation == null) {
+return;
+}
+String op = annotation.value();
+if (op == null) {
+return;
+}
+
+Map topoConf = null;
+if 
(containerRequestContext.getUriInfo().getPathParameters().containsKey("id")) {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+try {
+topoConf = (Map) 
JSONValue.parse(nimbusClient.getClient().getTopologyConf(
+
containerRequestContext.getUriInfo().getPathParameters().get("id").get(0)));
+} catch (TException e) {
+e.printStackTrace();
--- End diff --

Can we turn this into an exception instead? 


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-30 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r206236992
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/exceptionmappers/TExceptionMapper.java
 ---
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.daemon.ui.exceptionmappers;
+
+import java.util.HashMap;
+import java.util.Map;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.thrift.TException;
+import org.json.simple.JSONValue;
+
+@Provider
+public class TExceptionMapper implements ExceptionMapper {
+
+/**
+ * getResponse.
+ * @param ex ex
+ * @param responseStatus responseStatus
+ * @return getResponse
+ */
+public static Response getResponse(Exception ex, Response.Status 
responseStatus) {
+Response.ResponseBuilder builder = Response.status(responseStatus);
+Map body = new HashMap<>();
+body.put("error", ex.toString());
+body.put("errorMessage", ex.toString());
+return 
builder.entity(JSONValue.toJSONString(body)).type("application/json").build();
+}
+
+/**
+ * getResponse.
+ * @param ex ex
+ * @return getResponse
+ */
+public static Response getResponse(AuthorizationException ex) {
+Response.ResponseBuilder builder = 
Response.status(Response.Status.UNAUTHORIZED);
+Map body = new HashMap<>();
+body.put("error", ex.toString());
+body.put("errorMessage", ex.get_msg());
+return 
builder.entity(JSONValue.toJSONString(body)).type("application/json").build();
--- End diff --

Can we combine this with the other `getResponse`?

```
public static Response getResponse(AuthorizationException ex) {
return getResponse(ex, Response.Status.UNAUTHORIZED);
}
```


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-30 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r206240201
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/ui/resources/StormApiResource.java
 ---
@@ -0,0 +1,702 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.daemon.ui.resources;
+
+import com.codahale.metrics.Meter;
+import java.net.URLDecoder;
+import java.util.Map;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+
+import org.apache.storm.daemon.ui.UIHelpers;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private HttpServletRequest servletRequest;
+
+public static Map config = Utils.readStormConfig();
+
+public static Meter clusterConfigurationRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-cluster-configuration-http-requests");
+
+public static Meter clusterSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-cluster-summary-http-requests");
+
+public static Meter nimbusSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-nimbus-summary-http-requests");
+
+public static Meter supervisorRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-supervisor-http-requests");
+
+public static Meter supervisorSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-supervisor-summary-http-requests");
+
+public static Meter allTopologiesSummaryRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-all-topologies-summary-http-requests");
+
+public static Meter topologyPageRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-topology-page-http-requests");
+
+public static Meter topologyMetricRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-topology-metric-http-requests");
+
+public static Meter buildVisualizationRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-build-visualization-http-requests");
+
+public static Meter mkVisualizationDataRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-mk-visualization-data-http-requests");
+
+public static Meter componentPageRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-component-page-http-requests");
+
+public static Meter logConfigRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-log-config-http-requests");
+
+public static Meter activateTopologyRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-activate-topology-http-requests");
+
+public static Meter deactivateTopologyRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-deactivate-topology-http-requests");
+
+public static Meter debugTopologyRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-debug-topology-http-requests");
+
+public static Meter componentOpResponseRequestMeter =
+

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-27 Thread govind-menon
Github user govind-menon commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r205816093
  
--- Diff: pom.xml ---
@@ -329,7 +329,7 @@
 3.1.0
 1.0
 0.13.1
-2.24.1
+2.27
--- End diff --

This is the latest stable version. Specifically it fixes a bug where a 
filter was not being called before a method


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-27 Thread govind-menon
Github user govind-menon commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r205815906
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/filters/AuthorizedUserFilter.java ---
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.filters;
+
+import java.io.IOException;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.PathSegment;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.resources.NimbusOp;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+try {
+uiImpersonationHandler = StormCommon.mkAuthorizationHandler(
+(String) 
conf.get(DaemonConfig.NIMBUS_IMPERSONATION_AUTHORIZER), conf);
+uiAclHandler = StormCommon.mkAuthorizationHandler(
+(String) conf.get(DaemonConfig.NIMBUS_AUTHORIZER), 
conf);
+} catch (IllegalAccessException | InstantiationException | 
ClassNotFoundException e) {
+LOG.error("Error initializing AuthorizedUserFilter: ", e);
+throw new RuntimeException(e);
+}
+}
+
+@Override
+public void filter(ContainerRequestContext containerRequestContext) 
throws IOException {
+NimbusOp annotation = 
resourceInfo.getResourceMethod().getAnnotation(NimbusOp.class);
+if (annotation == null) {
+return;
+}
+String op = annotation.value();
+if (op == null) {
+return;
+}
+
+Map topoConf = null;
+if 
(containerRequestContext.getUriInfo().getPathParameters().containsKey("id")) {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+try {
+topoConf = (Map) 
JSONValue.parse(nimbusClient.getClient().getTopologyConf(
+
containerRequestContext.getUriInfo().getPathParameters().get("id").get(0)));
+} catch (TException e) {
+e.printStackTrace();
+}
+}
+
+ReqContext reqContext = ReqContext.context();
+
+if (reqContext.isImpersonating()) {
+if (uiImpersonationHandler != null) {
+if (!uiImpersonationHandler.permit(reqContext, op, 
topoConf)) {
+Principal realPrincipal = reqContext.realPrincipal();
+Principal principal = reqContext.principal();
+String user = "unknown";
+if (principal != null) {
+user = principal.getName();
+ 

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-27 Thread govind-menon
Github user govind-menon commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r205815985
  
--- Diff: storm-core/src/jvm/org/apache/storm/utils/Monitor.java ---
@@ -1,12 +1,18 @@
 /**
- * Licensed to the Apache Software Foundation (ASF) under one or more 
contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  
The ASF licenses this file to you under the Apache License, Version
- * 2.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)
--- End diff --

I mistakenly did this for fixing checkstyle


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204259473
  
--- Diff: storm-core/src/clj/org/apache/storm/ui/helpers.clj ---
@@ -56,7 +56,7 @@
 
 ;; TODO this function and its callings will be replace when ui.core and 
logviewer move to Java
 (defnk json-response
-  [data callback :need-serialize true :status 200 :headers {}]
-  {:status status
+   [data callback :need-serialize true :status 200 :headers {}]
--- End diff --

Same here.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204276809
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/filters/AuthorizedUserFilter.java ---
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.filters;
+
+import java.io.IOException;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.PathSegment;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.resources.NimbusOp;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+try {
+uiImpersonationHandler = StormCommon.mkAuthorizationHandler(
+(String) 
conf.get(DaemonConfig.NIMBUS_IMPERSONATION_AUTHORIZER), conf);
+uiAclHandler = StormCommon.mkAuthorizationHandler(
+(String) conf.get(DaemonConfig.NIMBUS_AUTHORIZER), 
conf);
+} catch (IllegalAccessException | InstantiationException | 
ClassNotFoundException e) {
+LOG.error("Error initializing AuthorizedUserFilter: ", e);
+throw new RuntimeException(e);
+}
+}
+
+@Override
+public void filter(ContainerRequestContext containerRequestContext) 
throws IOException {
+NimbusOp annotation = 
resourceInfo.getResourceMethod().getAnnotation(NimbusOp.class);
+if (annotation == null) {
+return;
+}
+String op = annotation.value();
+if (op == null) {
+return;
+}
+
+Map topoConf = null;
+if 
(containerRequestContext.getUriInfo().getPathParameters().containsKey("id")) {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+try {
+topoConf = (Map) 
JSONValue.parse(nimbusClient.getClient().getTopologyConf(
+
containerRequestContext.getUriInfo().getPathParameters().get("id").get(0)));
+} catch (TException e) {
+e.printStackTrace();
+}
+}
+
+ReqContext reqContext = ReqContext.context();
+
+if (reqContext.isImpersonating()) {
+if (uiImpersonationHandler != null) {
+if (!uiImpersonationHandler.permit(reqContext, op, 
topoConf)) {
+Principal realPrincipal = reqContext.realPrincipal();
+Principal principal = reqContext.principal();
+String user = "unknown";
+if (principal != null) {
+user = principal.getName();
+  

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204258860
  
--- Diff: external/storm-opentsdb/pom.xml ---
@@ -36,7 +36,7 @@
 
 
 
-2.23
+2.27
--- End diff --

If we would want to sync the jersey version to root pom, this property 
would not be needed.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204260307
  
--- Diff: 
storm-dist/binary/storm-webapp-bin/src/main/assembly/storm-webapp.xml ---
@@ -27,6 +27,18 @@
 false
 lib-webapp
 false
+
--- End diff --

Once we also move UI to webapp this will be no longer needed at all.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204262254
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java ---
@@ -1,12 +1,18 @@
 /**
- * Licensed to the Apache Software Foundation (ASF) under one or more 
contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  
The ASF licenses this file to you under the Apache License, Version
- * 2.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)
--- End diff --

Is it for fixing checkstyle, or unneeded change?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204259623
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/daemon/drpc/webapp/ReqContextFilter.java ---
@@ -28,10 +28,14 @@
 import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.container.PreMatching;
+import javax.ws.rs.ext.Provider;
 
 import org.apache.storm.security.auth.IHttpCredentialsPlugin;
 import org.apache.storm.security.auth.ReqContext;
 
+@Provider
+@PreMatching
 public class ReqContextFilter implements Filter {
--- End diff --

Once UI also leverages this class, I feel we would be better to have common 
package on webservice, and move this class to there.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204262044
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/filters/HeaderResponseServletFilter.java 
---
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.ui.filters;
+
+import com.codahale.metrics.Meter;
+import org.apache.storm.Constants;
+import org.apache.storm.metric.StormMetricsRegistry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+
+public class HeaderResponseServletFilter implements Filter {
+public static final Logger LOG = 
LoggerFactory.getLogger(HeaderResponseServletFilter.class);
+
+public static Meter webRequestMeter =
+StormMetricsRegistry.registerMeter("num-web-requests");
+
+public static Meter mainPageRequestMeter =
+
StormMetricsRegistry.registerMeter("ui:num-main-page-http-requests");
+@Override
+public void init(FilterConfig filterConfig) throws ServletException {
+
+}
+
+@Override
+public void doFilter(ServletRequest servletRequest, ServletResponse 
servletResponse, FilterChain filterChain) throws IOException, ServletException {
+webRequestMeter.mark();
+HttpServletRequest httpRequest = (HttpServletRequest) 
servletRequest;
+HttpServletResponse httpResponse = (HttpServletResponse) 
servletResponse;
+if ((httpRequest.getPathInfo()).equals("/index.html")) {
--- End diff --

It looks like porting `request-middleware` to filter and the conditions 
between `request-middleware` and here are not same.


https://github.com/apache/storm/blob/a7e817bcd1424d300ab5bad06c5d9f4729d9f347/storm-core/src/clj/org/apache/storm/ui/helpers.clj#L39-L55


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204259460
  
--- Diff: storm-core/src/clj/org/apache/storm/ui/core.clj ---
@@ -1613,24 +1613,24 @@
   https-want-client-auth (conf UI-HTTPS-WANT-CLIENT-AUTH)
   https-need-client-auth (conf UI-HTTPS-NEED-CLIENT-AUTH)]
   (StormMetricsRegistry/startMetricsReporters conf)
-  (UIHelpers/stormRunJetty  (int (conf UI-PORT))
-(conf UI-HOST)
-https-port
-header-buffer-size
-(reify IConfigurator
+  (UIHelpers/stormRunJetty (int (conf UI-PORT))
+   (conf UI-HOST)
+   https-port
+   header-buffer-size
+   (reify IConfigurator
   (execute [this server]
 (UIHelpers/configSsl server
-  https-port
-  https-ks-path
-  https-ks-password
-  https-ks-type
-  https-key-password
-  https-ts-path
-  https-ts-password
-  https-ts-type
-  https-need-client-auth
-  https-want-client-auth
-  header-buffer-size)
+ https-port
+ https-ks-path
+ https-ks-password
+ https-ks-type
+ https-key-password
+ https-ts-path
+ https-ts-password
+ https-ts-type
+ 
https-need-client-auth
+ 
https-want-client-auth
+ 
header-buffer-size)
--- End diff --

I think we will remove clj files later, but let's revert unneeded changes 
so that we can see smaller set of change.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204259546
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/command/UploadCredentials.java ---
@@ -34,6 +34,11 @@
 
 private static final Logger LOG = 
LoggerFactory.getLogger(UploadCredentials.class);
 
+/**
+ * Uploads credentials for a topology.
+ * @param args To accept topology name.
+ * @throws Exception Ignored.
--- End diff --

Same here.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204259528
  
--- Diff: storm-core/src/jvm/org/apache/storm/command/GetErrors.java ---
@@ -25,6 +25,11 @@
 import org.json.simple.JSONValue;
 
 public class GetErrors {
+/**
+ * Only get errors for a topology.
+ * @param args Used to accept the topology name.
+ * @throws Exception Ignored.
--- End diff --

rewrite this to `@throws Exception on errors` on consistency with 
BasicDrpcClient, or just remove `throws Exception`.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204258951
  
--- Diff: storm-client/src/jvm/org/apache/storm/Constants.java ---
@@ -55,5 +55,7 @@
 public static final String COMMON_ONHEAP_MEMORY_RESOURCE_NAME = 
"onheap.memory.mb";
 public static final String COMMON_OFFHEAP_MEMORY_RESOURCE_NAME = 
"offheap.memory.mb";
 public static final String COMMON_TOTAL_MEMORY_RESOURCE_NAME = 
"memory.mb";
+
+public static final String STORM_API_URL_PREFIX = "/api/v1/";
--- End diff --

I can only see the usage from UIServer (storm-core, ideally needs to be 
moved to webserver module), and I'm not sure client side needs to know about 
this.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204259185
  
--- Diff: storm-core/pom.xml ---
@@ -32,6 +32,9 @@
 
 /etc/storm
 
${project.build.directory}/native/worker-launcher
+2.27
+2.4.3
+1.19.4
--- End diff --

I'd like to see this placed in web-app or individual module before merging 
this in. Once we pull this in and release, follow-up issues would be easy to 
forget. And our final goal for breaking down `storm-core` is to remove 
`storm-core`, so moving this to web-app also helps on reducing the work on this.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204258845
  
--- Diff: pom.xml ---
@@ -329,7 +329,7 @@
 3.1.0
 1.0
 0.13.1
-2.24.1
+2.27
--- End diff --

Could you elaborate why we upgrade the version here?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204276473
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/filters/AuthorizedUserFilter.java ---
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.filters;
+
+import java.io.IOException;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.PathSegment;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.resources.NimbusOp;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+try {
+uiImpersonationHandler = StormCommon.mkAuthorizationHandler(
+(String) 
conf.get(DaemonConfig.NIMBUS_IMPERSONATION_AUTHORIZER), conf);
+uiAclHandler = StormCommon.mkAuthorizationHandler(
+(String) conf.get(DaemonConfig.NIMBUS_AUTHORIZER), 
conf);
+} catch (IllegalAccessException | InstantiationException | 
ClassNotFoundException e) {
+LOG.error("Error initializing AuthorizedUserFilter: ", e);
+throw new RuntimeException(e);
+}
+}
+
+@Override
+public void filter(ContainerRequestContext containerRequestContext) 
throws IOException {
+NimbusOp annotation = 
resourceInfo.getResourceMethod().getAnnotation(NimbusOp.class);
+if (annotation == null) {
+return;
+}
+String op = annotation.value();
+if (op == null) {
+return;
+}
+
+Map topoConf = null;
+if 
(containerRequestContext.getUriInfo().getPathParameters().containsKey("id")) {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+try {
+topoConf = (Map) 
JSONValue.parse(nimbusClient.getClient().getTopologyConf(
+
containerRequestContext.getUriInfo().getPathParameters().get("id").get(0)));
+} catch (TException e) {
+e.printStackTrace();
+}
+}
+
+ReqContext reqContext = ReqContext.context();
+
+if (reqContext.isImpersonating()) {
+if (uiImpersonationHandler != null) {
+if (!uiImpersonationHandler.permit(reqContext, op, 
topoConf)) {
+Principal realPrincipal = reqContext.realPrincipal();
+Principal principal = reqContext.principal();
+String user = "unknown";
+if (principal != null) {
+user = principal.getName();
+  

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204260283
  
--- Diff: 
storm-webapp/src/main/java/org/apache/storm/daemon/logviewer/handler/LogviewerLogPageHandler.java
 ---
@@ -37,7 +37,6 @@
 import static java.util.stream.Collectors.toList;
 import static org.apache.commons.lang.StringEscapeUtils.escapeHtml;
 
-import j2html.TagCreator;
--- End diff --

Are these for removing unused import?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204260022
  
--- Diff: storm-core/src/jvm/org/apache/storm/utils/Monitor.java ---
@@ -1,12 +1,18 @@
 /**
- * Licensed to the Apache Software Foundation (ASF) under one or more 
contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.  
The ASF licenses this file to you under the Apache License, Version
- * 2.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)
--- End diff --

Could you explain why this change is necessary?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204276964
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/exceptionmappers/NotAliveExceptionMapper.java
 ---
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.exceptionmappers;
+
+import org.apache.storm.generated.NotAliveException;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+@Provider
+public class NotAliveExceptionMapper implements 
ExceptionMapper {
+public Response toResponse(NotAliveException ex) {
+return 
Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
--- End diff --

I'd rather not accept unless the response is at least similar detail on 
current. (This also means current response is not a good format IMHO, and I 
don't want to make it worse.)


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204276808
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/filters/AuthorizedUserFilter.java ---
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.filters;
+
+import java.io.IOException;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.PathSegment;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.resources.NimbusOp;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+try {
+uiImpersonationHandler = StormCommon.mkAuthorizationHandler(
+(String) 
conf.get(DaemonConfig.NIMBUS_IMPERSONATION_AUTHORIZER), conf);
+uiAclHandler = StormCommon.mkAuthorizationHandler(
+(String) conf.get(DaemonConfig.NIMBUS_AUTHORIZER), 
conf);
+} catch (IllegalAccessException | InstantiationException | 
ClassNotFoundException e) {
+LOG.error("Error initializing AuthorizedUserFilter: ", e);
+throw new RuntimeException(e);
+}
+}
+
+@Override
+public void filter(ContainerRequestContext containerRequestContext) 
throws IOException {
+NimbusOp annotation = 
resourceInfo.getResourceMethod().getAnnotation(NimbusOp.class);
+if (annotation == null) {
+return;
+}
+String op = annotation.value();
+if (op == null) {
+return;
+}
+
+Map topoConf = null;
+if 
(containerRequestContext.getUriInfo().getPathParameters().containsKey("id")) {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+try {
+topoConf = (Map) 
JSONValue.parse(nimbusClient.getClient().getTopologyConf(
+
containerRequestContext.getUriInfo().getPathParameters().get("id").get(0)));
+} catch (TException e) {
+e.printStackTrace();
+}
+}
+
+ReqContext reqContext = ReqContext.context();
+
+if (reqContext.isImpersonating()) {
+if (uiImpersonationHandler != null) {
+if (!uiImpersonationHandler.permit(reqContext, op, 
topoConf)) {
+Principal realPrincipal = reqContext.realPrincipal();
+Principal principal = reqContext.principal();
+String user = "unknown";
+if (principal != null) {
+user = principal.getName();
+  

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-22 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r204259143
  
--- Diff: storm-core/pom.xml ---
@@ -32,6 +32,9 @@
 
 /etc/storm
 
${project.build.directory}/native/worker-launcher
+2.27
--- End diff --

Can we rely on root pom version of jersey?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-13 Thread danny0405
Github user danny0405 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r202504021
  
--- Diff: storm-core/src/clj/org/apache/storm/ui/core.clj ---
@@ -1613,24 +1613,24 @@
   https-want-client-auth (conf UI-HTTPS-WANT-CLIENT-AUTH)
   https-need-client-auth (conf UI-HTTPS-NEED-CLIENT-AUTH)]
   (StormMetricsRegistry/startMetricsReporters conf)
-  (UIHelpers/stormRunJetty  (int (conf UI-PORT))
-(conf UI-HOST)
-https-port
-header-buffer-size
-(reify IConfigurator
+  (UIHelpers/stormRunJetty (int (conf UI-PORT))
+   (conf UI-HOST)
+   https-port
+   header-buffer-size
+   (reify IConfigurator
   (execute [this server]
 (UIHelpers/configSsl server
-  https-port
-  https-ks-path
-  https-ks-password
-  https-ks-type
-  https-key-password
-  https-ts-path
-  https-ts-password
-  https-ts-type
-  https-need-client-auth
-  https-want-client-auth
-  header-buffer-size)
+ https-port
+ https-ks-path
+ https-ks-password
+ https-ks-type
+ https-key-password
+ https-ts-path
+ https-ts-password
+ https-ts-type
+ 
https-need-client-auth
+ 
https-want-client-auth
+ 
header-buffer-size)
--- End diff --

This file has many nits whitespace.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-11 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r201738893
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/exceptionmappers/AuthorizationExceptionMapper.java
 ---
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.exceptionmappers;
+
+import org.apache.storm.generated.AuthorizationException;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+@Provider
+public class AuthorizationExceptionMapper implements 
ExceptionMapper {
+public Response toResponse(AuthorizationException ex) {
+return 
Response.status(javax.ws.rs.core.Response.Status.UNAUTHORIZED).build();
--- End diff --

nit: you don't need the `javax.ws.rs.core.` here. 

```
return Response.status(Response.Status.UNAUTHORIZED).build();
```

should work


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-11 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r201739243
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/exceptionmappers/NotAliveExceptionMapper.java
 ---
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.exceptionmappers;
+
+import org.apache.storm.generated.NotAliveException;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+@Provider
+public class NotAliveExceptionMapper implements 
ExceptionMapper {
+public Response toResponse(NotAliveException ex) {
+return 
Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
--- End diff --

Could you file a follow on JIRA to handle this better in both the response 
and in the UI?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-11 Thread govind-menon
Github user govind-menon commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r201717965
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java ---
@@ -295,9 +391,1104 @@ public static String getJsonResponseBody(Object 
data, String callback, boolean n
 return callback != null ? wrapJsonInCallback(callback, 
serializedData) : serializedData;
 }
 
+/**
+ * Converts exception into json map.
+ * @param ex Exception to be converted.
+ * @param statusCode Status code to be returned.
+ * @return Map to be converted into json.
+ */
 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());
+return ImmutableMap.of(
+"error", statusCode
++ " "
++ HttpStatus.getMessage(statusCode),
+"errorMessage", sw.toString());
+}
+
+/**
+ * Converts thrift call result into map fit for UI/api.
+ * @param clusterSummary Obtained from Nimbus.
+ * @param user User Making request
+ * @param conf Storm Conf
+ * @return Cluster Summary for display on UI/monitoring purposes via 
API
+ */
+public static Map getClusterSummary(ClusterSummary 
clusterSummary, String user,
+Map conf) {
+Map result = new HashMap();
+List supervisorSummaries = 
clusterSummary.get_supervisors();
+List topologySummaries = 
clusterSummary.get_topologies();
+
+Integer usedSlots =
+supervisorSummaries.stream().mapToInt(
+SupervisorSummary::get_num_used_workers).sum();
+Integer totalSlots =
+supervisorSummaries.stream().mapToInt(
+SupervisorSummary::get_num_workers).sum();
+
+Integer totalTasks =
+topologySummaries.stream().mapToInt(
+TopologySummary::get_num_tasks).sum();
+Integer totalExecutors =
+topologySummaries.stream().mapToInt(
+TopologySummary::get_num_executors).sum();
+
+Double supervisorTotalMemory =
+supervisorSummaries.stream().mapToDouble(x -> 
x.get_total_resources().getOrDefault(
+Constants.COMMON_TOTAL_MEMORY_RESOURCE_NAME,
+
x.get_total_resources().get(Config.SUPERVISOR_MEMORY_CAPACITY_MB)
+)
+).sum();
+
+Double supervisorTotalCpu =
+supervisorSummaries.stream().mapToDouble(x -> 
x.get_total_resources().getOrDefault(
+Constants.COMMON_CPU_RESOURCE_NAME,
+
x.get_total_resources().get(Config.SUPERVISOR_CPU_CAPACITY)
+)
+).sum();
+
+Double supervisorUsedMemory =
+
supervisorSummaries.stream().mapToDouble(SupervisorSummary::get_used_mem).sum();
+Double supervisorUsedCpu =
+
supervisorSummaries.stream().mapToDouble(SupervisorSummary::get_used_cpu).sum();
+Double supervisorFragementedCpu =
+supervisorSummaries.stream().mapToDouble(
+SupervisorSummary::get_fragmented_cpu).sum();
+Double supervisorFragmentedMem =
+supervisorSummaries.stream().mapToDouble(
+SupervisorSummary::get_fragmented_mem).sum();
+
+
+result.put("user", user);
+result.put("stormVersion", VersionInfo.getVersion());
+result.put("supervisors", supervisorSummaries.size());
+result.put("topologies", clusterSummary.get_topologies_size());
+result.put("slotsUsed", usedSlots);
+result.put("slotsTotal", totalSlots);
+result.put("slotsFree", totalSlots - usedSlots);
+result.put("tasksTotal", totalTasks);
+result.put("totalExecutors", totalExecutors);
+
+result.put("totalMem", supervisorTotalMemory);
+result.put("totalCpu", supervisorTotalCpu);
+result.put("availMem", supervisorTotalMemory - 
supervisorUsedMemory);
+result.put("availCpu", supervisorTotalCpu - supervisorUsedCpu);
+result.put("fragmentedMem", supervisorFragmentedMem);
+result.put("fragmentedCpu", supervisorFragementedCpu);
+result.put("schedulerDisplayResource",
+  

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-11 Thread govind-menon
Github user govind-menon commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r201717748
  
--- Diff: storm-core/test/jvm/org/apache/storm/ui/StormApiResourceTest.java 
---
@@ -0,0 +1,19 @@
+package org.apache.storm.ui;
+
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.client.WebTarget;
+
+import org.glassfish.grizzly.http.server.HttpServer;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+
+public class StormApiResourceTest {
+
+private HttpServer server;
+private WebTarget target;
--- End diff --

I will add a single test here


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-11 Thread govind-menon
Github user govind-menon commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r201717619
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/filters/AuthorizedUserFilter.java ---
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.filters;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.PathSegment;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+public static Map resourceMethodToNimbusOps = new 
HashMap();
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+resourceMethodToNimbusOps.put("getClusterSummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getNimbusSummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getOwnerResources", 
"getOwnerResourceSummaries");
+resourceMethodToNimbusOps.put("getOwnerResource", 
"getOwnerResourceSummaries");
+resourceMethodToNimbusOps.put("getSupervisorSummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getSupervisor", 
"getSupervisorPageInfo");
+resourceMethodToNimbusOps.put("getTopologySummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getTopology", "getTopology");
+resourceMethodToNimbusOps.put("getTopologyMetrics", "getTopology");
+resourceMethodToNimbusOps.put("getTopologyVisializationInit", 
"getTopology");
+resourceMethodToNimbusOps.put("getTopologyVisualization", 
"getTopology");
+resourceMethodToNimbusOps.put("getTopologyLogconfig", 
"getTopology");
+resourceMethodToNimbusOps.put("putTopologyLogconfig", 
"setLogConfig");
+resourceMethodToNimbusOps.put("putTopologyActivate", "activate");
+resourceMethodToNimbusOps.put("putTopologyDeactivate", 
"deactivate");
+resourceMethodToNimbusOps.put("putTopologyDebugActionSpct", 
"debug");
+
resourceMethodToNimbusOps.put("putTopologyComponentDebugActionSpct", "debug");
+resourceMethodToNimbusOps.put("putTopologyRebalance", "rebalance");
+resourceMethodToNimbusOps.put("putTopologyKill", "killTopology");
+resourceMethodToNimbusOps.put("getTopologyProfilingStart", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingStop", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingDumpProfile", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingRestartWorker", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingDumpheap", 
"setWorkerProfiler");
--- End diff --

Still working on this


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-11 Thread govind-menon
Github user govind-menon commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r201717451
  
--- Diff: storm-client/src/jvm/org/apache/storm/Constants.java ---
@@ -55,5 +55,7 @@
 public static final String COMMON_ONHEAP_MEMORY_RESOURCE_NAME = 
"onheap.memory.mb";
 public static final String COMMON_OFFHEAP_MEMORY_RESOURCE_NAME = 
"offheap.memory.mb";
 public static final String COMMON_TOTAL_MEMORY_RESOURCE_NAME = 
"memory.mb";
+
+public static final String STORM_API_URL_PREFIX = "/api/v1/";
--- End diff --

It's now used in the StormApiResource


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-11 Thread govind-menon
Github user govind-menon commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r201717347
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIServer.java ---
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.drpc.webapp.ReqContextFilter;
+import org.apache.storm.security.auth.IHttpCredentialsPlugin;
+import org.apache.storm.security.auth.ServerAuthUtils;
+import org.apache.storm.ui.filters.AuthorizedUserFilter;
+import org.apache.storm.utils.Utils;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.servlet.ServletContainer;
+
+
+import javax.servlet.DispatcherType;
+import java.util.EnumSet;
+import java.util.Map;
+
+/**
+ * Main class.
+ *
+ */
+public class UIServer {
+
+public static void addRequestContextFilter(ServletContextHandler 
context,
+   String configName, 
Map conf) {
+IHttpCredentialsPlugin auth = 
ServerAuthUtils.getHttpCredentialsPlugin(conf, (String) conf.get(configName));
+ReqContextFilter filter = new ReqContextFilter(auth);
+context.addFilter(new FilterHolder(filter), "/*", 
EnumSet.allOf(DispatcherType.class));
+}
+
+public static void main(String[] args) throws Exception {
+
+Map conf = Utils.readStormConfig();
+Server jettyServer = new Server();
+ServerConnector connector = new ServerConnector(jettyServer);
+connector.setPort(4443);
+//connector.setPort((Integer) conf.get(DaemonConfig.UI_PORT));
+jettyServer.addConnector(connector);
+
+ServletContextHandler context = new 
ServletContextHandler(ServletContextHandler.SESSIONS);
+context.setContextPath("/");
+jettyServer.setHandler(context);
+
+
+ResourceConfig resourceConfig =
+new ResourceConfig()
+.packages("org.apache.storm.ui.resources")
+.register(AuthorizedUserFilter.class);
+
+ServletHolder jerseyServlet = new ServletHolder(new 
ServletContainer(resourceConfig));
+jerseyServlet.setInitOrder(0);
+context.addServlet(jerseyServlet, "/api/v1/*");
+addRequestContextFilter(context, 
DaemonConfig.DRPC_HTTP_CREDS_PLUGIN, conf);
+
+// add special pathspec of static content mapped to the homePath
+ServletHolder holderHome = new ServletHolder("static-home", 
DefaultServlet.class);
+
holderHome.setInitParameter("resourceBase",UIServer.class.getProtectionDomain().getCodeSource().getLocation().toExternalForm()
 + "/WEB-INF/");
+holderHome.setInitParameter("dirAllowed","true");
+holderHome.setInitParameter("pathInfoOnly","true");
+context.addServlet(holderHome,"/*");
+
+
+// Lastly, the default servlet for root content (always needed, to 
satisfy servlet spec)
+ServletHolder holderPwd = new ServletHolder("default", 
DefaultServlet.class);
+holderPwd.setInitParameter("dirAllowed","true");
--- End diff --

I checked and it does not allow that


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200435146
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/resources/StormApiResource.java ---
@@ -0,0 +1,741 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.UIHelpers;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private UriInfo info;
+
+@Context
+private HttpServletRequest servletRequest;
+
+
+@Context
+private ServletContext servletContext;
+
+public static Map config = Utils.readStormConfig();
+/**
+ * /api/v1/cluster/configuration -> nimbus configuration.
+ */
+
+@GET
+@Path("/cluster/configuration")
+@Produces("application/json")
+public Response getClusterConfiguration() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+return 
Response.status(Response.Status.OK).entity(nimbusClient.getClient().getNimbusConf()).build();
+} catch (AuthorizationException e) {
+e.printStackTrace();
+return Response.status(Response.Status.UNAUTHORIZED).build();
+} catch (TException e) {
+e.printStackTrace();
+return 
Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
+}
+}
+
+/**
+ * /api/v1/cluster/summary -> cluster summary.
+ */
+@GET
+@Path("/cluster/summary")
+@Produces("application/json")
+public Response getClusterSummary() {
+try {
--- End diff --

Could you add back in the metrics for each of the http calls?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200439312
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java ---
@@ -295,9 +391,1104 @@ public static String getJsonResponseBody(Object 
data, String callback, boolean n
 return callback != null ? wrapJsonInCallback(callback, 
serializedData) : serializedData;
 }
 
+/**
+ * Converts exception into json map.
+ * @param ex Exception to be converted.
+ * @param statusCode Status code to be returned.
+ * @return Map to be converted into json.
+ */
 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());
+return ImmutableMap.of(
+"error", statusCode
++ " "
++ HttpStatus.getMessage(statusCode),
+"errorMessage", sw.toString());
+}
+
+/**
+ * Converts thrift call result into map fit for UI/api.
+ * @param clusterSummary Obtained from Nimbus.
+ * @param user User Making request
+ * @param conf Storm Conf
+ * @return Cluster Summary for display on UI/monitoring purposes via 
API
+ */
+public static Map getClusterSummary(ClusterSummary 
clusterSummary, String user,
+Map conf) {
+Map result = new HashMap();
+List supervisorSummaries = 
clusterSummary.get_supervisors();
+List topologySummaries = 
clusterSummary.get_topologies();
+
+Integer usedSlots =
+supervisorSummaries.stream().mapToInt(
+SupervisorSummary::get_num_used_workers).sum();
+Integer totalSlots =
+supervisorSummaries.stream().mapToInt(
+SupervisorSummary::get_num_workers).sum();
+
+Integer totalTasks =
+topologySummaries.stream().mapToInt(
+TopologySummary::get_num_tasks).sum();
+Integer totalExecutors =
+topologySummaries.stream().mapToInt(
+TopologySummary::get_num_executors).sum();
+
+Double supervisorTotalMemory =
+supervisorSummaries.stream().mapToDouble(x -> 
x.get_total_resources().getOrDefault(
+Constants.COMMON_TOTAL_MEMORY_RESOURCE_NAME,
+
x.get_total_resources().get(Config.SUPERVISOR_MEMORY_CAPACITY_MB)
+)
+).sum();
+
+Double supervisorTotalCpu =
+supervisorSummaries.stream().mapToDouble(x -> 
x.get_total_resources().getOrDefault(
+Constants.COMMON_CPU_RESOURCE_NAME,
+
x.get_total_resources().get(Config.SUPERVISOR_CPU_CAPACITY)
+)
+).sum();
+
+Double supervisorUsedMemory =
+
supervisorSummaries.stream().mapToDouble(SupervisorSummary::get_used_mem).sum();
+Double supervisorUsedCpu =
+
supervisorSummaries.stream().mapToDouble(SupervisorSummary::get_used_cpu).sum();
+Double supervisorFragementedCpu =
+supervisorSummaries.stream().mapToDouble(
+SupervisorSummary::get_fragmented_cpu).sum();
+Double supervisorFragmentedMem =
+supervisorSummaries.stream().mapToDouble(
+SupervisorSummary::get_fragmented_mem).sum();
+
+
+result.put("user", user);
+result.put("stormVersion", VersionInfo.getVersion());
+result.put("supervisors", supervisorSummaries.size());
+result.put("topologies", clusterSummary.get_topologies_size());
+result.put("slotsUsed", usedSlots);
+result.put("slotsTotal", totalSlots);
+result.put("slotsFree", totalSlots - usedSlots);
+result.put("tasksTotal", totalTasks);
+result.put("totalExecutors", totalExecutors);
+
+result.put("totalMem", supervisorTotalMemory);
+result.put("totalCpu", supervisorTotalCpu);
+result.put("availMem", supervisorTotalMemory - 
supervisorUsedMemory);
+result.put("availCpu", supervisorTotalCpu - supervisorUsedCpu);
+result.put("fragmentedMem", supervisorFragmentedMem);
+result.put("fragmentedCpu", supervisorFragementedCpu);
+result.put("schedulerDisplayResource",
+   

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200476039
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/resources/StormApiResource.java ---
@@ -0,0 +1,741 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.UIHelpers;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private UriInfo info;
+
+@Context
+private HttpServletRequest servletRequest;
+
+
+@Context
+private ServletContext servletContext;
+
+public static Map config = Utils.readStormConfig();
+/**
+ * /api/v1/cluster/configuration -> nimbus configuration.
+ */
+
+@GET
+@Path("/cluster/configuration")
+@Produces("application/json")
+public Response getClusterConfiguration() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+return 
Response.status(Response.Status.OK).entity(nimbusClient.getClient().getNimbusConf()).build();
+} catch (AuthorizationException e) {
+e.printStackTrace();
+return Response.status(Response.Status.UNAUTHORIZED).build();
+} catch (TException e) {
+e.printStackTrace();
+return 
Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
+}
+}
+
+/**
+ * /api/v1/cluster/summary -> cluster summary.
+ */
+@GET
+@Path("/cluster/summary")
+@Produces("application/json")
+public Response getClusterSummary() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+String user = servletRequest.getRemoteUser();
+return 
Response.status(Response.Status.OK).entity(JSONValue.toJSONString(
+UIHelpers.getClusterSummary(
+nimbusClient.getClient().getClusterInfo(), 
user, config))).build();
+} catch (AuthorizationException e) {
+e.printStackTrace();
+return Response.status(Response.Status.UNAUTHORIZED).build();
+
+} catch (TException e) {
+e.printStackTrace();
+return 
Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
+}
+}
+
+/**
+ * /api/v1/nimbus/summary -> nimbus summary.
+ */
+@GET
+@Path("/nimbus/summary")
+@Produces("application/json")
+public Response getNimbusSummary() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+String user = servletRequest.getRemoteUser();
+return 

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200437014
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java ---
@@ -295,9 +391,1104 @@ public static String getJsonResponseBody(Object 
data, String callback, boolean n
 return callback != null ? wrapJsonInCallback(callback, 
serializedData) : serializedData;
 }
 
+/**
+ * Converts exception into json map.
+ * @param ex Exception to be converted.
+ * @param statusCode Status code to be returned.
+ * @return Map to be converted into json.
+ */
 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());
+return ImmutableMap.of(
+"error", statusCode
++ " "
++ HttpStatus.getMessage(statusCode),
+"errorMessage", sw.toString());
+}
+
+/**
+ * Converts thrift call result into map fit for UI/api.
+ * @param clusterSummary Obtained from Nimbus.
+ * @param user User Making request
+ * @param conf Storm Conf
+ * @return Cluster Summary for display on UI/monitoring purposes via 
API
+ */
+public static Map getClusterSummary(ClusterSummary 
clusterSummary, String user,
+Map conf) {
+Map result = new HashMap();
+List supervisorSummaries = 
clusterSummary.get_supervisors();
+List topologySummaries = 
clusterSummary.get_topologies();
+
+Integer usedSlots =
+supervisorSummaries.stream().mapToInt(
+SupervisorSummary::get_num_used_workers).sum();
+Integer totalSlots =
+supervisorSummaries.stream().mapToInt(
+SupervisorSummary::get_num_workers).sum();
+
+Integer totalTasks =
+topologySummaries.stream().mapToInt(
+TopologySummary::get_num_tasks).sum();
+Integer totalExecutors =
+topologySummaries.stream().mapToInt(
+TopologySummary::get_num_executors).sum();
+
+Double supervisorTotalMemory =
--- End diff --

nit: same for `double` vs `Double` here.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200474709
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/resources/StormApiResource.java ---
@@ -0,0 +1,741 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.UIHelpers;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private UriInfo info;
+
+@Context
+private HttpServletRequest servletRequest;
+
+
+@Context
+private ServletContext servletContext;
+
+public static Map config = Utils.readStormConfig();
+/**
+ * /api/v1/cluster/configuration -> nimbus configuration.
+ */
+
+@GET
+@Path("/cluster/configuration")
+@Produces("application/json")
+public Response getClusterConfiguration() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+return 
Response.status(Response.Status.OK).entity(nimbusClient.getClient().getNimbusConf()).build();
+} catch (AuthorizationException e) {
+e.printStackTrace();
+return Response.status(Response.Status.UNAUTHORIZED).build();
+} catch (TException e) {
+e.printStackTrace();
+return 
Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
+}
+}
+
+/**
+ * /api/v1/cluster/summary -> cluster summary.
+ */
+@GET
+@Path("/cluster/summary")
+@Produces("application/json")
+public Response getClusterSummary() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+String user = servletRequest.getRemoteUser();
+return 
Response.status(Response.Status.OK).entity(JSONValue.toJSONString(
+UIHelpers.getClusterSummary(
+nimbusClient.getClient().getClusterInfo(), 
user, config))).build();
+} catch (AuthorizationException e) {
+e.printStackTrace();
+return Response.status(Response.Status.UNAUTHORIZED).build();
+
+} catch (TException e) {
+e.printStackTrace();
+return 
Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
+}
+}
+
+/**
+ * /api/v1/nimbus/summary -> nimbus summary.
+ */
+@GET
+@Path("/nimbus/summary")
+@Produces("application/json")
+public Response getNimbusSummary() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+String user = servletRequest.getRemoteUser();
+return 

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200440315
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/resources/StormApiResource.java ---
@@ -0,0 +1,741 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.UIHelpers;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private UriInfo info;
+
+@Context
+private HttpServletRequest servletRequest;
+
+
+@Context
+private ServletContext servletContext;
+
+public static Map config = Utils.readStormConfig();
+/**
+ * /api/v1/cluster/configuration -> nimbus configuration.
+ */
+
+@GET
+@Path("/cluster/configuration")
+@Produces("application/json")
+public Response getClusterConfiguration() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+return 
Response.status(Response.Status.OK).entity(nimbusClient.getClient().getNimbusConf()).build();
--- End diff --

We are not handling the callback query parameter on any of the APIs, but it 
is everywhere in the old REST calls.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200436771
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java ---
@@ -295,9 +391,1104 @@ public static String getJsonResponseBody(Object 
data, String callback, boolean n
 return callback != null ? wrapJsonInCallback(callback, 
serializedData) : serializedData;
 }
 
+/**
+ * Converts exception into json map.
+ * @param ex Exception to be converted.
+ * @param statusCode Status code to be returned.
+ * @return Map to be converted into json.
+ */
 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());
+return ImmutableMap.of(
+"error", statusCode
++ " "
++ HttpStatus.getMessage(statusCode),
+"errorMessage", sw.toString());
+}
+
+/**
+ * Converts thrift call result into map fit for UI/api.
+ * @param clusterSummary Obtained from Nimbus.
+ * @param user User Making request
+ * @param conf Storm Conf
+ * @return Cluster Summary for display on UI/monitoring purposes via 
API
+ */
+public static Map getClusterSummary(ClusterSummary 
clusterSummary, String user,
+Map conf) {
+Map result = new HashMap();
+List supervisorSummaries = 
clusterSummary.get_supervisors();
+List topologySummaries = 
clusterSummary.get_topologies();
+
+Integer usedSlots =
--- End diff --

nit: Please make all of these `int` and not `Integer`.  `sum()` returns an 
`int` and we are not auto-boxing it to an `Integer` in multiple places, so it 
is going to be a bit cleaner (imho).


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200454747
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java ---
@@ -295,9 +391,1104 @@ public static String getJsonResponseBody(Object 
data, String callback, boolean n
 return callback != null ? wrapJsonInCallback(callback, 
serializedData) : serializedData;
 }
 
+/**
+ * Converts exception into json map.
+ * @param ex Exception to be converted.
+ * @param statusCode Status code to be returned.
+ * @return Map to be converted into json.
+ */
 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());
+return ImmutableMap.of(
+"error", statusCode
++ " "
++ HttpStatus.getMessage(statusCode),
+"errorMessage", sw.toString());
+}
+
+/**
+ * Converts thrift call result into map fit for UI/api.
+ * @param clusterSummary Obtained from Nimbus.
+ * @param user User Making request
+ * @param conf Storm Conf
+ * @return Cluster Summary for display on UI/monitoring purposes via 
API
+ */
+public static Map getClusterSummary(ClusterSummary 
clusterSummary, String user,
+Map conf) {
+Map result = new HashMap();
+List supervisorSummaries = 
clusterSummary.get_supervisors();
+List topologySummaries = 
clusterSummary.get_topologies();
+
+Integer usedSlots =
+supervisorSummaries.stream().mapToInt(
+SupervisorSummary::get_num_used_workers).sum();
+Integer totalSlots =
+supervisorSummaries.stream().mapToInt(
+SupervisorSummary::get_num_workers).sum();
+
+Integer totalTasks =
+topologySummaries.stream().mapToInt(
+TopologySummary::get_num_tasks).sum();
+Integer totalExecutors =
+topologySummaries.stream().mapToInt(
+TopologySummary::get_num_executors).sum();
+
+Double supervisorTotalMemory =
+supervisorSummaries.stream().mapToDouble(x -> 
x.get_total_resources().getOrDefault(
+Constants.COMMON_TOTAL_MEMORY_RESOURCE_NAME,
+
x.get_total_resources().get(Config.SUPERVISOR_MEMORY_CAPACITY_MB)
+)
+).sum();
+
+Double supervisorTotalCpu =
+supervisorSummaries.stream().mapToDouble(x -> 
x.get_total_resources().getOrDefault(
+Constants.COMMON_CPU_RESOURCE_NAME,
+
x.get_total_resources().get(Config.SUPERVISOR_CPU_CAPACITY)
+)
+).sum();
+
+Double supervisorUsedMemory =
+
supervisorSummaries.stream().mapToDouble(SupervisorSummary::get_used_mem).sum();
+Double supervisorUsedCpu =
+
supervisorSummaries.stream().mapToDouble(SupervisorSummary::get_used_cpu).sum();
+Double supervisorFragementedCpu =
+supervisorSummaries.stream().mapToDouble(
+SupervisorSummary::get_fragmented_cpu).sum();
+Double supervisorFragmentedMem =
+supervisorSummaries.stream().mapToDouble(
+SupervisorSummary::get_fragmented_mem).sum();
+
+
+result.put("user", user);
+result.put("stormVersion", VersionInfo.getVersion());
+result.put("supervisors", supervisorSummaries.size());
+result.put("topologies", clusterSummary.get_topologies_size());
+result.put("slotsUsed", usedSlots);
+result.put("slotsTotal", totalSlots);
+result.put("slotsFree", totalSlots - usedSlots);
+result.put("tasksTotal", totalTasks);
+result.put("totalExecutors", totalExecutors);
+
+result.put("totalMem", supervisorTotalMemory);
+result.put("totalCpu", supervisorTotalCpu);
+result.put("availMem", supervisorTotalMemory - 
supervisorUsedMemory);
+result.put("availCpu", supervisorTotalCpu - supervisorUsedCpu);
+result.put("fragmentedMem", supervisorFragmentedMem);
+result.put("fragmentedCpu", supervisorFragementedCpu);
+result.put("schedulerDisplayResource",
+   

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200415886
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/filters/AuthorizedUserFilter.java ---
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.filters;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.PathSegment;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+public static Map resourceMethodToNimbusOps = new 
HashMap();
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+resourceMethodToNimbusOps.put("getClusterSummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getNimbusSummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getOwnerResources", 
"getOwnerResourceSummaries");
+resourceMethodToNimbusOps.put("getOwnerResource", 
"getOwnerResourceSummaries");
+resourceMethodToNimbusOps.put("getSupervisorSummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getSupervisor", 
"getSupervisorPageInfo");
+resourceMethodToNimbusOps.put("getTopologySummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getTopology", "getTopology");
+resourceMethodToNimbusOps.put("getTopologyMetrics", "getTopology");
+resourceMethodToNimbusOps.put("getTopologyVisializationInit", 
"getTopology");
+resourceMethodToNimbusOps.put("getTopologyVisualization", 
"getTopology");
+resourceMethodToNimbusOps.put("getTopologyLogconfig", 
"getTopology");
+resourceMethodToNimbusOps.put("putTopologyLogconfig", 
"setLogConfig");
+resourceMethodToNimbusOps.put("putTopologyActivate", "activate");
+resourceMethodToNimbusOps.put("putTopologyDeactivate", 
"deactivate");
+resourceMethodToNimbusOps.put("putTopologyDebugActionSpct", 
"debug");
+
resourceMethodToNimbusOps.put("putTopologyComponentDebugActionSpct", "debug");
+resourceMethodToNimbusOps.put("putTopologyRebalance", "rebalance");
+resourceMethodToNimbusOps.put("putTopologyKill", "killTopology");
+resourceMethodToNimbusOps.put("getTopologyProfilingStart", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingStop", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingDumpProfile", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingRestartWorker", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingDumpheap", 
"setWorkerProfiler");
+try {
+uiImpersonationHandler = StormCommon.mkAuthorizationHandler(
+(String) 

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200433651
  
--- Diff: storm-core/test/jvm/org/apache/storm/ui/StormApiResourceTest.java 
---
@@ -0,0 +1,19 @@
+package org.apache.storm.ui;
+
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.client.WebTarget;
+
+import org.glassfish.grizzly.http.server.HttpServer;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+
+public class StormApiResourceTest {
+
+private HttpServer server;
+private WebTarget target;
--- End diff --

Is this just a place holder for a test or are we actually going to get some 
tests?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200407479
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java ---
@@ -77,6 +140,11 @@ public static String prettyUptimeStr(String val, 
Object[][] dividers) {
 return Joiner.on(" ").join(tmp);
 }
 
+/**
+ * See above javadoc.
--- End diff --

nit: This takes as long to write as `Prettify uptime string.` so can we 
just use it instead?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200403357
  
--- Diff: conf/defaults.yaml ---
@@ -88,7 +88,7 @@ nimbus.topology.blobstore.deletion.delay.ms: 30
 
 ### ui.* configs are for the master
 ui.host: 0.0.0.0
-ui.port: 8080
+ui.port: 4443
--- End diff --

I don't think we need to change the default UI port.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200408980
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIServer.java ---
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.drpc.webapp.ReqContextFilter;
+import org.apache.storm.security.auth.IHttpCredentialsPlugin;
+import org.apache.storm.security.auth.ServerAuthUtils;
+import org.apache.storm.ui.filters.AuthorizedUserFilter;
+import org.apache.storm.utils.Utils;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.servlet.ServletContainer;
+
+
+import javax.servlet.DispatcherType;
+import java.util.EnumSet;
+import java.util.Map;
+
+/**
+ * Main class.
+ *
+ */
+public class UIServer {
+
+public static void addRequestContextFilter(ServletContextHandler 
context,
+   String configName, 
Map conf) {
+IHttpCredentialsPlugin auth = 
ServerAuthUtils.getHttpCredentialsPlugin(conf, (String) conf.get(configName));
+ReqContextFilter filter = new ReqContextFilter(auth);
+context.addFilter(new FilterHolder(filter), "/*", 
EnumSet.allOf(DispatcherType.class));
+}
+
+public static void main(String[] args) throws Exception {
+
+Map conf = Utils.readStormConfig();
+Server jettyServer = new Server();
+ServerConnector connector = new ServerConnector(jettyServer);
+connector.setPort(4443);
+//connector.setPort((Integer) conf.get(DaemonConfig.UI_PORT));
+jettyServer.addConnector(connector);
+
+ServletContextHandler context = new 
ServletContextHandler(ServletContextHandler.SESSIONS);
+context.setContextPath("/");
+jettyServer.setHandler(context);
+
+
+ResourceConfig resourceConfig =
+new ResourceConfig()
+.packages("org.apache.storm.ui.resources")
+.register(AuthorizedUserFilter.class);
+
+ServletHolder jerseyServlet = new ServletHolder(new 
ServletContainer(resourceConfig));
+jerseyServlet.setInitOrder(0);
+context.addServlet(jerseyServlet, "/api/v1/*");
+addRequestContextFilter(context, 
DaemonConfig.DRPC_HTTP_CREDS_PLUGIN, conf);
+
+// add special pathspec of static content mapped to the homePath
+ServletHolder holderHome = new ServletHolder("static-home", 
DefaultServlet.class);
+
holderHome.setInitParameter("resourceBase",UIServer.class.getProtectionDomain().getCodeSource().getLocation().toExternalForm()
 + "/WEB-INF/");
+holderHome.setInitParameter("dirAllowed","true");
+holderHome.setInitParameter("pathInfoOnly","true");
+context.addServlet(holderHome,"/*");
+
+
+// Lastly, the default servlet for root content (always needed, to 
satisfy servlet spec)
+ServletHolder holderPwd = new ServletHolder("default", 
DefaultServlet.class);
+holderPwd.setInitParameter("dirAllowed","true");
+context.addServlet(holderPwd,"/");
+
+try {
+jettyServer.start();
+jettyServer.join();
+} catch (Throwable t) {
+t.printStackTrace(System.err);
--- End diff --

Can we please just let the exception be thrown by main instead?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200404317
  
--- Diff: storm-core/pom.xml ---
@@ -194,10 +249,7 @@
 java-hamcrest
 test
 
-
-javax.servlet
-javax.servlet-api
-
+
--- End diff --

Not sure why we need the comment, if we removed the dependency.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200408787
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIServer.java ---
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.drpc.webapp.ReqContextFilter;
+import org.apache.storm.security.auth.IHttpCredentialsPlugin;
+import org.apache.storm.security.auth.ServerAuthUtils;
+import org.apache.storm.ui.filters.AuthorizedUserFilter;
+import org.apache.storm.utils.Utils;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.servlet.ServletContainer;
+
+
+import javax.servlet.DispatcherType;
+import java.util.EnumSet;
+import java.util.Map;
+
+/**
+ * Main class.
+ *
+ */
+public class UIServer {
+
+public static void addRequestContextFilter(ServletContextHandler 
context,
+   String configName, 
Map conf) {
+IHttpCredentialsPlugin auth = 
ServerAuthUtils.getHttpCredentialsPlugin(conf, (String) conf.get(configName));
+ReqContextFilter filter = new ReqContextFilter(auth);
+context.addFilter(new FilterHolder(filter), "/*", 
EnumSet.allOf(DispatcherType.class));
+}
+
+public static void main(String[] args) throws Exception {
+
+Map conf = Utils.readStormConfig();
+Server jettyServer = new Server();
+ServerConnector connector = new ServerConnector(jettyServer);
+connector.setPort(4443);
+//connector.setPort((Integer) conf.get(DaemonConfig.UI_PORT));
+jettyServer.addConnector(connector);
+
+ServletContextHandler context = new 
ServletContextHandler(ServletContextHandler.SESSIONS);
+context.setContextPath("/");
+jettyServer.setHandler(context);
+
+
+ResourceConfig resourceConfig =
+new ResourceConfig()
+.packages("org.apache.storm.ui.resources")
+.register(AuthorizedUserFilter.class);
+
+ServletHolder jerseyServlet = new ServletHolder(new 
ServletContainer(resourceConfig));
+jerseyServlet.setInitOrder(0);
+context.addServlet(jerseyServlet, "/api/v1/*");
+addRequestContextFilter(context, 
DaemonConfig.DRPC_HTTP_CREDS_PLUGIN, conf);
+
+// add special pathspec of static content mapped to the homePath
+ServletHolder holderHome = new ServletHolder("static-home", 
DefaultServlet.class);
+
holderHome.setInitParameter("resourceBase",UIServer.class.getProtectionDomain().getCodeSource().getLocation().toExternalForm()
 + "/WEB-INF/");
+holderHome.setInitParameter("dirAllowed","true");
+holderHome.setInitParameter("pathInfoOnly","true");
+context.addServlet(holderHome,"/*");
+
+
+// Lastly, the default servlet for root content (always needed, to 
satisfy servlet spec)
+ServletHolder holderPwd = new ServletHolder("default", 
DefaultServlet.class);
+holderPwd.setInitParameter("dirAllowed","true");
--- End diff --

question: does dirAllowed let someone do a '/foo/../../bar/' like URL?  I 
assume that it does not, but I want to be sure.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200403856
  
--- Diff: storm-core/pom.xml ---
@@ -32,6 +32,9 @@
 
 /etc/storm
 
${project.build.directory}/native/worker-launcher
+2.27
+2.4.3
+1.19.4
--- End diff --

At some point can we move all of this to the web-app, or split the UI off 
into it's own module if we need to.  This can be a follow on JIRA.  I just 
don't see a reason to keep it here when it no longer needs clojure.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200430165
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/resources/StormApiResource.java ---
@@ -0,0 +1,741 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.UIHelpers;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private UriInfo info;
+
+@Context
+private HttpServletRequest servletRequest;
+
+
+@Context
+private ServletContext servletContext;
+
+public static Map config = Utils.readStormConfig();
+/**
+ * /api/v1/cluster/configuration -> nimbus configuration.
+ */
+
+@GET
+@Path("/cluster/configuration")
+@Produces("application/json")
+public Response getClusterConfiguration() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
--- End diff --

Please put the NimbusClient in the try so it is closed when we are done 
with it instead of waiting for GC? and we don't have to re-read the stom config 
for each request.

```
try (NimbusClient nimbusClient = NimbusClient.getConfiguredClient(config)) {
return Response.status(Response.Status.OK)...
}
```


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200406284
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java ---
@@ -59,6 +116,12 @@
 new Object[]{ "d", null }
 };
 
+/**
+ * Prettigy uptime string.
--- End diff --

nit: prettify?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200407111
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/UIHelpers.java ---
@@ -59,6 +116,12 @@
 new Object[]{ "d", null }
 };
 
+/**
+ * Prettigy uptime string.
+ * @param val val.
--- End diff --

nit: val is a stringified integer.  Why it is not just an int?  but that is 
for a follow on JIRA not here.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200405995
  
--- Diff: storm-core/src/jvm/org/apache/storm/ui/StormUiApplication.java ---
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.HashSet;
+import java.util.Set;
+import javax.ws.rs.ApplicationPath;
+import javax.ws.rs.core.Application;
+
+import org.apache.storm.ui.resources.StormApiResource;
+// import org.apache.storm.ui.resources.StormUiResource;
--- End diff --

I assume that these are going to be un-commented once we enable serving the 
UI itself?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200429329
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/resources/StormApiResource.java ---
@@ -0,0 +1,741 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.UIHelpers;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private UriInfo info;
+
+@Context
+private HttpServletRequest servletRequest;
+
+
+@Context
+private ServletContext servletContext;
+
+public static Map config = Utils.readStormConfig();
+/**
+ * /api/v1/cluster/configuration -> nimbus configuration.
+ */
+
+@GET
+@Path("/cluster/configuration")
+@Produces("application/json")
+public Response getClusterConfiguration() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+return 
Response.status(Response.Status.OK).entity(nimbusClient.getClient().getNimbusConf()).build();
+} catch (AuthorizationException e) {
--- End diff --

Nit: could we look at using an ExceptionMapper instead?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200429402
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/resources/StormApiResource.java ---
@@ -0,0 +1,741 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.UIHelpers;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private UriInfo info;
+
+@Context
+private HttpServletRequest servletRequest;
+
+
+@Context
+private ServletContext servletContext;
+
+public static Map config = Utils.readStormConfig();
+/**
+ * /api/v1/cluster/configuration -> nimbus configuration.
+ */
+
+@GET
+@Path("/cluster/configuration")
+@Produces("application/json")
+public Response getClusterConfiguration() {
+try {
+NimbusClient nimbusClient = 
NimbusClient.getConfiguredClient(Utils.readStormConfig());
+return 
Response.status(Response.Status.OK).entity(nimbusClient.getClient().getNimbusConf()).build();
+} catch (AuthorizationException e) {
+e.printStackTrace();
+return Response.status(Response.Status.UNAUTHORIZED).build();
+} catch (TException e) {
+e.printStackTrace();
--- End diff --

Could we log the exception instead of just printing the stack trace?


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200431722
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/resources/StormApiResource.java ---
@@ -0,0 +1,741 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.UIHelpers;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private UriInfo info;
--- End diff --

This is not used.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200403478
  
--- Diff: storm-client/src/jvm/org/apache/storm/Constants.java ---
@@ -55,5 +55,7 @@
 public static final String COMMON_ONHEAP_MEMORY_RESOURCE_NAME = 
"onheap.memory.mb";
 public static final String COMMON_OFFHEAP_MEMORY_RESOURCE_NAME = 
"offheap.memory.mb";
 public static final String COMMON_TOTAL_MEMORY_RESOURCE_NAME = 
"memory.mb";
+
+public static final String STORM_API_URL_PREFIX = "/api/v1/";
--- End diff --

This is never used.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200432708
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/resources/StormApiResource.java ---
@@ -0,0 +1,741 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.resources;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.ui.UIHelpers;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Root resource (exposed at "storm" path).
+ */
+@Path("/")
+public class StormApiResource {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(StormApiResource.class);
+
+@Context
+private UriInfo info;
+
+@Context
+private HttpServletRequest servletRequest;
+
+
+@Context
+private ServletContext servletContext;
--- End diff --

This is not being used.


---


[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/2752#discussion_r200418105
  
--- Diff: 
storm-core/src/jvm/org/apache/storm/ui/filters/AuthorizedUserFilter.java ---
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.filters;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.Principal;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringJoiner;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.PathSegment;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.storm.DaemonConfig;
+import org.apache.storm.daemon.StormCommon;
+import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.security.auth.IAuthorizer;
+import org.apache.storm.security.auth.ReqContext;
+import org.apache.storm.thrift.TException;
+import org.apache.storm.utils.NimbusClient;
+import org.apache.storm.utils.Utils;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Provider
+public class AuthorizedUserFilter implements ContainerRequestFilter {
+
+public static final Logger LOG = 
LoggerFactory.getLogger(AuthorizedUserFilter.class);
+public static Map conf = Utils.readStormConfig();
+public static IAuthorizer uiImpersonationHandler;
+public static IAuthorizer uiAclHandler;
+
+public static Map resourceMethodToNimbusOps = new 
HashMap();
+
+@Context private ResourceInfo resourceInfo;
+
+static {
+resourceMethodToNimbusOps.put("getClusterSummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getNimbusSummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getOwnerResources", 
"getOwnerResourceSummaries");
+resourceMethodToNimbusOps.put("getOwnerResource", 
"getOwnerResourceSummaries");
+resourceMethodToNimbusOps.put("getSupervisorSummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getSupervisor", 
"getSupervisorPageInfo");
+resourceMethodToNimbusOps.put("getTopologySummary", 
"getClusterInfo");
+resourceMethodToNimbusOps.put("getTopology", "getTopology");
+resourceMethodToNimbusOps.put("getTopologyMetrics", "getTopology");
+resourceMethodToNimbusOps.put("getTopologyVisializationInit", 
"getTopology");
+resourceMethodToNimbusOps.put("getTopologyVisualization", 
"getTopology");
+resourceMethodToNimbusOps.put("getTopologyLogconfig", 
"getTopology");
+resourceMethodToNimbusOps.put("putTopologyLogconfig", 
"setLogConfig");
+resourceMethodToNimbusOps.put("putTopologyActivate", "activate");
+resourceMethodToNimbusOps.put("putTopologyDeactivate", 
"deactivate");
+resourceMethodToNimbusOps.put("putTopologyDebugActionSpct", 
"debug");
+
resourceMethodToNimbusOps.put("putTopologyComponentDebugActionSpct", "debug");
+resourceMethodToNimbusOps.put("putTopologyRebalance", "rebalance");
+resourceMethodToNimbusOps.put("putTopologyKill", "killTopology");
+resourceMethodToNimbusOps.put("getTopologyProfilingStart", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingStop", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingDumpProfile", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingRestartWorker", 
"setWorkerProfiler");
+resourceMethodToNimbusOps.put("getTopologyProfilingDumpheap", 
"setWorkerProfiler");
--- End diff --

Could we add in a custom annotation to the APIs instead of having a 
separate map here that is highly coupled to the 

[GitHub] storm pull request #2752: Storm 1311 Migration of UI from clj to Java

2018-07-05 Thread govind-menon
GitHub user govind-menon opened a pull request:

https://github.com/apache/storm/pull/2752

Storm 1311 Migration of UI from clj to Java

This PR basically duplicates the entire UI core in Java as an embedded 
Jetty Server. The follow on PR will delete the clj code and substitute the cli 
command (I'm separating it so they can be compared more easily).

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/govind-menon/storm STORM-1311-Jetty

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/storm/pull/2752.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #2752


commit 35409496e24b76e6d5f628da0b399582b8a70d8b
Author: Govind Menon 
Date:   2018-05-29T19:57:41Z

STORM-1311: UI Migration from clj to java

commit fc6498cac5136bb5d29d140ff1b411c2205a38e0
Author: Govind Menon 
Date:   2018-06-01T17:41:35Z

STORM-1311: Using Jetty instead of Grizzly

commit dec452f51a61a4e8e787666d11ddd4cc5e9fee30
Author: Govind Menon 
Date:   2018-06-27T05:17:50Z

STORM-1311: Adding all GET routes

commit 75529bb8cb7db7b1cdd50445db675e5d5741d66e
Author: Govind Menon 
Date:   2018-06-27T15:30:40Z

STORM-1311: Leftover from merge

commit 2f5893ba1bf252f769a2b47c41315bd3bd1414a8
Author: Govind Menon 
Date:   2018-07-05T08:20:31Z

STORM-1311: Added rest of puts and made static file serving configuration 
generic

commit 66ebc86e4214f4cfce3c67b3080737805d72e312
Author: Govind Menon 
Date:   2018-07-05T08:43:18Z

STORM-1311: Cleaning up POM files




---