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

[1/4] storm git commit: Add metric to track when TGT expires

Repository: storm
Updated Branches:
  refs/heads/master c02f4a0db -> 5f0573983


Add metric to track when TGT expires


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/49ab628a
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/49ab628a
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/49ab628a

Branch: refs/heads/master
Commit: 49ab628a952a58c236ad31d93a939ab75794e131
Parents: abf0f8a
Author: Aaron Gresch <ag...@yahoo-inc.com>
Authored: Mon Feb 5 14:56:13 2018 -0600
Committer: Aaron Gresch <ag...@yahoo-inc.com>
Committed: Mon Feb 5 14:56:13 2018 -0600

----------------------------------------------------------------------
 .../org/apache/storm/daemon/worker/Worker.java  | 25 ++++---
 .../apache/storm/daemon/worker/WorkerState.java | 45 +++++++-----
 .../storm/executor/bolt/BoltExecutor.java       | 23 +++++-
 .../org/apache/storm/metric/api/IMetric.java    | 14 +++-
 .../storm/security/INimbusCredentialPlugin.java |  7 +-
 .../storm/security/auth/kerberos/AutoTGT.java   | 76 +++++++++++++-------
 6 files changed, 127 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/49ab628a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
index 519e7ce..4209422 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
@@ -18,6 +18,9 @@
 
 package org.apache.storm.daemon.worker;
 
+import com.google.common.base.Preconditions;
+import com.lmax.disruptor.EventHandler;
+
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.Charset;
@@ -62,20 +65,17 @@ import org.apache.storm.security.auth.AuthUtils;
 import org.apache.storm.security.auth.IAutoCredentials;
 import org.apache.storm.stats.StatsUtil;
 import org.apache.storm.utils.ConfigUtils;
-import org.apache.storm.utils.Utils;
 import org.apache.storm.utils.DisruptorBackpressureCallback;
 import org.apache.storm.utils.LocalState;
 import org.apache.storm.utils.ObjectReader;
 import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
 import org.apache.storm.utils.WorkerBackpressureCallback;
 import org.apache.storm.utils.WorkerBackpressureThread;
 import org.apache.zookeeper.data.ACL;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-import com.lmax.disruptor.EventHandler;
-
 import uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
 
 public class Worker implements Shutdownable, DaemonCommon {
@@ -105,7 +105,7 @@ public class Worker implements Shutdownable, DaemonCommon {
     /**
      * TODO: should worker even take the topologyId as input? this should be
      * deducible from cluster state (by searching through assignments)
-     * what about if there's inconsistency in assignments? -> but nimbus should guarantee this consistency
+     * what about if there's inconsistency in assignments? -> but nimbus should guarantee this consistency.
      *
      * @param conf         - Storm configuration
      * @param context      -
@@ -159,7 +159,7 @@ public class Worker implements Shutdownable, DaemonCommon {
             @Override public Object run() throws Exception {
                 workerState =
                     new WorkerState(conf, context, topologyId, assignmentId, port, workerId, topologyConf, stateStorage,
-                        stormClusterState);
+                        stormClusterState, autoCreds);
 
                 // Heartbeat here so that worker process dies if this fails
                 // it's important that worker heartbeat to supervisor ASAP so that supervisor knows
@@ -328,7 +328,8 @@ public class Worker implements Shutdownable, DaemonCommon {
 
     public Map<String, Long> getCurrentBlobVersions() throws IOException {
         Map<String, Long> results = new HashMap<>();
-        Map<String, Map<String, Object>> blobstoreMap = (Map<String, Map<String, Object>>) workerState.getTopologyConf().get(Config.TOPOLOGY_BLOBSTORE_MAP);
+        Map<String, Map<String, Object>> blobstoreMap =
+                (Map<String, Map<String, Object>>) workerState.getTopologyConf().get(Config.TOPOLOGY_BLOBSTORE_MAP);
         if (blobstoreMap != null) {
             String stormRoot = ConfigUtils.supervisorStormDistRoot(workerState.getTopologyConf(), workerState.getTopologyId());
             for (Map.Entry<String, Map<String, Object>> entry : blobstoreMap.entrySet()) {
@@ -368,7 +369,8 @@ public class Worker implements Shutdownable, DaemonCommon {
     }
 
     public void checkThrottleChanged() {
-        boolean throttleOn = workerState.stormClusterState.topologyBackpressure(topologyId, backpressureZnodeTimeoutMs, this::checkThrottleChanged);
+        boolean throttleOn =
+                workerState.stormClusterState.topologyBackpressure(topologyId, backpressureZnodeTimeoutMs, this::checkThrottleChanged);
         workerState.throttleOn.set(throttleOn);
     }
 
@@ -385,7 +387,7 @@ public class Worker implements Shutdownable, DaemonCommon {
 
     /**
      * make a handler for the worker's send disruptor queue to
-     * check highWaterMark and lowWaterMark for backpressure
+     * check highWaterMark and lowWaterMark for backpressure.
      */
     private DisruptorBackpressureCallback mkDisruptorBackpressureHandler(WorkerState workerState) {
         return new DisruptorBackpressureCallback() {
@@ -402,7 +404,7 @@ public class Worker implements Shutdownable, DaemonCommon {
     }
 
     /**
-     * make a handler that checks and updates worker's backpressure flag
+     * make a handler that checks and updates worker's backpressure flag.
      */
     private WorkerBackpressureCallback mkBackpressureHandler(Map<String, Object> topologyConf) {
         final List<IRunningExecutor> executors = executorsAtom.get();
@@ -432,7 +434,8 @@ public class Worker implements Shutdownable, DaemonCommon {
 
                     if (currBackpressureTimestamp != prevBackpressureTimestamp) {
                         try {
-                            LOG.debug("worker backpressure timestamp changing from {} to {}", prevBackpressureTimestamp, currBackpressureTimestamp);
+                            LOG.debug("worker backpressure timestamp changing from {} to {}",
+                                    prevBackpressureTimestamp, currBackpressureTimestamp);
                             stormClusterState.workerBackpressure(topologyId, assignmentId, (long) port, currBackpressureTimestamp);
                             // doing the local reset after the zk update succeeds is very important to avoid a bad state upon zk exception
                             workerState.backpressure.set(currBackpressureTimestamp);

http://git-wip-us.apache.org/repos/asf/storm/blob/49ab628a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
index ec2ff59..e238961 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
@@ -20,6 +20,17 @@ package org.apache.storm.daemon.worker;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.storm.Config;
 import org.apache.storm.Constants;
 import org.apache.storm.StormTimer;
@@ -47,32 +58,20 @@ import org.apache.storm.messaging.IConnection;
 import org.apache.storm.messaging.IContext;
 import org.apache.storm.messaging.TaskMessage;
 import org.apache.storm.messaging.TransportFactory;
+import org.apache.storm.security.auth.IAutoCredentials;
 import org.apache.storm.serialization.KryoTupleSerializer;
 import org.apache.storm.task.WorkerTopologyContext;
 import org.apache.storm.tuple.AddressedTuple;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.utils.ConfigUtils;
-import org.apache.storm.utils.Utils;
 import org.apache.storm.utils.DisruptorQueue;
 import org.apache.storm.utils.ObjectReader;
 import org.apache.storm.utils.ThriftTopologyUtils;
 import org.apache.storm.utils.TransferDrainer;
+import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Function;
-import java.util.stream.Collectors;
 
 public class WorkerState {
 
@@ -145,7 +144,9 @@ public class WorkerState {
         return componentToSortedTasks;
     }
 
-    public Map<String, Long> getBlobToLastKnownVersion() {return blobToLastKnownVersion;}
+    public Map<String, Long> getBlobToLastKnownVersion() {
+        return blobToLastKnownVersion;
+    }
 
     public AtomicReference<Map<NodeInfo, IConnection>> getCachedNodeToPortSocket() {
         return cachedNodeToPortSocket;
@@ -265,12 +266,15 @@ public class WorkerState {
     private final AtomicLong nextUpdate = new AtomicLong(0);
     private final boolean trySerializeLocal;
     private final TransferDrainer drainer;
+    private final Collection<IAutoCredentials> autoCredentials;
 
     private static final long LOAD_REFRESH_INTERVAL_MS = 5000L;
 
     public WorkerState(Map<String, Object> conf, IContext mqContext, String topologyId, String assignmentId, int port, String workerId,
-        Map<String, Object> topologyConf, IStateStorage stateStorage, IStormClusterState stormClusterState)
+                       Map<String, Object> topologyConf, IStateStorage stateStorage, IStormClusterState stormClusterState,
+                       Collection<IAutoCredentials> autoCredentials)
         throws IOException, InvalidTopologyException {
+        this.autoCredentials = autoCredentials;
         this.executors = new HashSet<>(readWorkerExecutors(stormClusterState, topologyId, assignmentId, port));
         this.transferQueue = new DisruptorQueue("worker-transfer-queue",
             ObjectReader.getInt(topologyConf.get(Config.TOPOLOGY_TRANSFER_BUFFER_SIZE)),
@@ -307,7 +311,8 @@ public class WorkerState {
         this.componentToStreamToFields = new HashMap<>();
         for (String c : ThriftTopologyUtils.getComponentIds(systemTopology)) {
             Map<String, Fields> streamToFields = new HashMap<>();
-            for (Map.Entry<String, StreamInfo> stream : ThriftTopologyUtils.getComponentCommon(systemTopology, c).get_streams().entrySet()) {
+            for (Map.Entry<String, StreamInfo> stream :
+                    ThriftTopologyUtils.getComponentCommon(systemTopology, c).get_streams().entrySet()) {
                 streamToFields.put(stream.getKey(), new Fields(stream.getValue().get_output_fields()));
             }
             componentToStreamToFields.put(c, streamToFields);
@@ -477,7 +482,7 @@ public class WorkerState {
 
     /**
      * we will wait all connections to be ready and then activate the spout/bolt
-     * when the worker bootup
+     * when the worker bootup.
      */
     public void activateWorkerWhenAllConnectionsReady() {
         int delaySecs = 0;
@@ -631,6 +636,10 @@ public class WorkerState {
             || ((ConnectionWithStatus) connection).status() == ConnectionWithStatus.Status.Ready;
     }
 
+    public Collection<IAutoCredentials> getAutoCredentials() {
+        return this.autoCredentials;
+    }
+
     private List<List<Long>> readWorkerExecutors(IStormClusterState stormClusterState, String topologyId, String assignmentId,
         int port) {
         LOG.info("Reading assignments");

http://git-wip-us.apache.org/repos/asf/storm/blob/49ab628a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
index 4e46dc5..01bff97 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
@@ -15,10 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.executor.bolt;
 
 import com.google.common.collect.ImmutableMap;
 import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import org.apache.storm.Config;
 import org.apache.storm.Constants;
 import org.apache.storm.ICredentialsListener;
 import org.apache.storm.daemon.Task;
@@ -26,6 +31,8 @@ import org.apache.storm.daemon.metrics.BuiltinMetricsUtil;
 import org.apache.storm.daemon.worker.WorkerState;
 import org.apache.storm.executor.Executor;
 import org.apache.storm.hooks.info.BoltExecuteInfo;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.security.auth.IAutoCredentials;
 import org.apache.storm.stats.BoltExecutorStats;
 import org.apache.storm.task.IBolt;
 import org.apache.storm.task.IOutputCollector;
@@ -33,14 +40,13 @@ import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.tuple.TupleImpl;
 import org.apache.storm.utils.ConfigUtils;
-import org.apache.storm.utils.Utils;
 import org.apache.storm.utils.DisruptorQueue;
 import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Map;
-import java.util.concurrent.Callable;
+
 
 public class BoltExecutor extends Executor {
 
@@ -77,6 +83,17 @@ public class BoltExecutor extends Executor {
                 Map cachedNodePortToSocket = (Map) workerData.getCachedNodeToPortSocket().get();
                 BuiltinMetricsUtil.registerIconnectionClientMetrics(cachedNodePortToSocket, topoConf, userContext);
                 BuiltinMetricsUtil.registerIconnectionServerMetric(workerData.getReceiver(), topoConf, userContext);
+
+                // add any autocredential expiry metrics from the worker
+                if (workerData.getAutoCredentials() != null) {
+                    int bucketSize = ((Number) topoConf.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)).intValue();
+                    for (IAutoCredentials autoCredential : workerData.getAutoCredentials()) {
+                        if (autoCredential instanceof IMetric) {
+                            IMetric metric = (IMetric)autoCredential;
+                            userContext.registerMetric(metric.getMetricName(), metric, bucketSize);
+                        }
+                    }
+                }
             } else {
                 Map<String, DisruptorQueue> map = ImmutableMap.of("sendqueue", sendQueue, "receive", receiveQueue);
                 BuiltinMetricsUtil.registerQueueMetrics(map, topoConf, userContext);

http://git-wip-us.apache.org/repos/asf/storm/blob/49ab628a/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
index 9e365c1..a08ac49 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
@@ -15,16 +15,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.storm.metric.api;
 
 /**
- * Produces metrics
+ * Produces metrics.
  */
 public interface IMetric {
     /**
-     * @return an object that will be sent sent to {@link IMetricsConsumer#handleDataPoints(org.apache.storm.metric.api.IMetricsConsumer.TaskInfo, java.util.Collection)}.
+     * @return an object that will be sent sent to {@link
+     * IMetricsConsumer#handleDataPoints(org.apache.storm.metric.api.IMetricsConsumer.TaskInfo, java.util.Collection)}.
      * If null is returned nothing will be sent.
      * If this value can be reset, like with a counter, a side effect of calling this should be that the value is reset.
      */
     public Object getValueAndReset();
+
+    /**
+     * Get the metric name.
+     * @return the name of the metric
+     */
+    default String getMetricName() {
+        return this.getClass().getSimpleName();
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/49ab628a/storm-client/src/jvm/org/apache/storm/security/INimbusCredentialPlugin.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/security/INimbusCredentialPlugin.java b/storm-client/src/jvm/org/apache/storm/security/INimbusCredentialPlugin.java
index cec005e..8882835 100644
--- a/storm-client/src/jvm/org/apache/storm/security/INimbusCredentialPlugin.java
+++ b/storm-client/src/jvm/org/apache/storm/security/INimbusCredentialPlugin.java
@@ -15,12 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.storm.security;
 
-import org.apache.storm.daemon.Shutdownable;
+package org.apache.storm.security;
 
 import java.util.Map;
-import org.apache.storm.generated.StormTopology;
+import org.apache.storm.daemon.Shutdownable;
 
 /**
  * Nimbus auto credential plugin that will be called on nimbus host
@@ -45,7 +44,7 @@ public interface INimbusCredentialPlugin extends Shutdownable {
      */
     @Deprecated
     default void populateCredentials(Map<String, String> credentials, Map<String, Object> topologyConf) {
-        throw new IllegalStateException("One of the populateCredentials methods must be overridden");
+        throw new IllegalStateException("One of the populateCredentials methods must be overridden by " + this);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/storm/blob/49ab628a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
index 4a4c52b..3c0a248 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
@@ -18,38 +18,38 @@
 
 package org.apache.storm.security.auth.kerberos;
 
-import org.apache.storm.security.auth.IAutoCredentials;
-import org.apache.storm.security.auth.ICredentialsRenewer;
-import org.apache.storm.security.auth.AuthUtils;
-
+import java.lang.reflect.Method;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
-import java.lang.reflect.Method;
-import java.lang.reflect.Constructor;
-import java.security.Principal;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.Iterator;
 
-import javax.security.auth.kerberos.KerberosTicket;
-import javax.security.auth.kerberos.KerberosPrincipal;
-import javax.security.auth.login.Configuration;
-import javax.security.auth.login.LoginContext;
 import javax.security.auth.DestroyFailedException;
 import javax.security.auth.RefreshFailedException;
 import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
 import javax.xml.bind.DatatypeConverter;
 
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.security.auth.AuthUtils;
+import org.apache.storm.security.auth.IAutoCredentials;
+import org.apache.storm.security.auth.ICredentialsRenewer;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * Automatically take a user's TGT, and push it, and renew it in Nimbus.
  */
-public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
+public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetric {
     private static final Logger LOG = LoggerFactory.getLogger(AutoTGT.class);
     private static final float TICKET_RENEW_WINDOW = 0.80f;
     protected static final AtomicReference<KerberosTicket> kerbTicket = new AtomicReference<>();
     private Map<String, Object> conf;
+    private Map<String, String> credentials;
 
     public void prepare(Map<String, Object> conf) {
         this.conf = conf;
@@ -57,7 +57,7 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
 
     private static KerberosTicket getTGT(Subject subject) {
         Set<KerberosTicket> tickets = subject.getPrivateCredentials(KerberosTicket.class);
-        for(KerberosTicket ticket: tickets) {
+        for (KerberosTicket ticket: tickets) {
             KerberosPrincipal server = ticket.getServer();
             if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) {
                 return ticket;
@@ -68,6 +68,7 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
 
     @Override
     public void populateCredentials(Map<String, String> credentials) {
+        this.credentials = credentials;
         //Log the user in and get the TGT
         try {
             Configuration login_conf = AuthUtils.GetConfiguration(conf);
@@ -83,7 +84,7 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
 
                 if (tgt == null) { //error
                     throw new RuntimeException("Fail to verify user principal with section \""
-                            +AuthUtils.LOGIN_CONTEXT_CLIENT+"\" in login configuration file "+ login_conf);
+                            + AuthUtils.LOGIN_CONTEXT_CLIENT + "\" in login configuration file " + login_conf);
                 }
 
                 if (!tgt.isForwardable()) {
@@ -94,7 +95,7 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
                     throw new RuntimeException("The TGT found is not renewable");
                 }
 
-                LOG.info("Pushing TGT for "+tgt.getClient()+" to topology.");
+                LOG.info("Pushing TGT for " + tgt.getClient() + " to topology.");
                 saveTGT(tgt, credentials);
             } finally {
                 lc.logout();
@@ -124,11 +125,13 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
 
     @Override
     public void updateSubject(Subject subject, Map<String, String> credentials) {
+        this.credentials = credentials;
         populateSubjectWithTGT(subject, credentials);
     }
 
     @Override
     public void populateSubject(Subject subject, Map<String, String> credentials) {
+        this.credentials = credentials;
         populateSubjectWithTGT(subject, credentials);
         loginHadoopUser(subject);
     }
@@ -146,7 +149,7 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
 
     public static void clearCredentials(Subject subject, KerberosTicket tgt) {
         Set<Object> creds = subject.getPrivateCredentials();
-        synchronized(creds) {
+        synchronized (creds) {
             Iterator<Object> iterator = creds.iterator();
             while (iterator.hasNext()) {
                 Object o = iterator.next();
@@ -160,7 +163,7 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
                     }
                 }
             }
-            if(tgt != null) {
+            if (tgt != null) {
                 creds.add(tgt);
             }
         }
@@ -182,16 +185,17 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
         try {
             Method isSecEnabled = ugi.getMethod("isSecurityEnabled");
             if (!((Boolean)isSecEnabled.invoke(null))) {
-                LOG.warn("Hadoop is on the classpath but not configured for " +
-                  "security, if you want security you need to be sure that " +
-                  "hadoop.security.authentication=kerberos in core-site.xml " +
-                  "in your jar");
+                LOG.warn("Hadoop is on the classpath but not configured for "
+                        + "security, if you want security you need to be sure that "
+                        + "hadoop.security.authentication=kerberos in core-site.xml "
+                        + "in your jar");
                 return;
             }
             Method login = ugi.getMethod("loginUserFromSubject", Subject.class);
             login.invoke(null, subject);
         } catch (Exception e) {
-            LOG.warn("Something went wrong while trying to initialize Hadoop through reflection. This version of hadoop may not be compatible.", e);
+            LOG.warn("Something went wrong while trying to initialize Hadoop through reflection. This version of hadoop "
+                    + "may not be compatible.", e);
         }
     }
 
@@ -203,13 +207,14 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
 
     @Override
     public void renew(Map<String,String> credentials, Map<String, Object> topologyConf, String topologyOwnerPrincipal) {
+        this.credentials = credentials;
         KerberosTicket tgt = getTGT(credentials);
         if (tgt != null) {
             long refreshTime = getRefreshTime(tgt);
             long now = System.currentTimeMillis();
             if (now >= refreshTime) {
                 try {
-                    LOG.info("Renewing TGT for "+tgt.getClient());
+                    LOG.info("Renewing TGT for " + tgt.getClient());
                     tgt.refresh();
                     saveTGT(tgt, credentials);
                 } catch (RefreshFailedException e) {
@@ -219,6 +224,25 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
         }
     }
 
+    private Long getMsecsUntilExpiration() {
+        KerberosTicket tgt = getTGT(this.credentials);
+        if (tgt == null) {
+            return null;
+        }
+        long end = tgt.getEndTime().getTime();
+        return end - System.currentTimeMillis();
+    }
+
+    @Override
+    public Object getValueAndReset() {
+        return this.getMsecsUntilExpiration();
+    }
+
+    @Override
+    public String getMetricName() {
+        return "TGT-TimeToExpiryMsecs";
+    }
+
     public static void main(String[] args) throws Exception {
         AutoTGT at = new AutoTGT();
         Map<String, Object> conf = new java.util.HashMap();
@@ -228,6 +252,8 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
         at.populateCredentials(creds);
         Subject s = new Subject();
         at.populateSubject(s, creds);
-        LOG.info("Got a Subject "+s);
+        LOG.info("Got a Subject " + s);
     }
+
+
 }


[3/4] storm git commit: remove iMetric implementation from AutoTGT

Posted by bo...@apache.org.
remove iMetric implementation from AutoTGT


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b5aca6ce
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b5aca6ce
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b5aca6ce

Branch: refs/heads/master
Commit: b5aca6ce0ef64df81868cbb60ac436f18e347a37
Parents: ab60951
Author: Aaron Gresch <ag...@yahoo-inc.com>
Authored: Wed Feb 7 08:39:49 2018 -0600
Committer: Aaron Gresch <ag...@yahoo-inc.com>
Committed: Wed Feb 7 08:39:49 2018 -0600

----------------------------------------------------------------------
 .../org/apache/storm/security/auth/kerberos/AutoTGT.java  | 10 ++--------
 1 file changed, 2 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b5aca6ce/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
index c0d40ad..1e69b5d 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
@@ -34,7 +34,6 @@ import javax.security.auth.login.LoginContext;
 import javax.xml.bind.DatatypeConverter;
 
 import org.apache.storm.Config;
-import org.apache.storm.metric.api.IMetric;
 import org.apache.storm.metric.api.IMetricsRegistrant;
 import org.apache.storm.security.auth.AuthUtils;
 import org.apache.storm.security.auth.IAutoCredentials;
@@ -47,7 +46,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Automatically take a user's TGT, and push it, and renew it in Nimbus.
  */
-public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetric, IMetricsRegistrant {
+public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetricsRegistrant {
     private static final Logger LOG = LoggerFactory.getLogger(AutoTGT.class);
     private static final float TICKET_RENEW_WINDOW = 0.80f;
     protected static final AtomicReference<KerberosTicket> kerbTicket = new AtomicReference<>();
@@ -236,11 +235,6 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetric,
         return end - System.currentTimeMillis();
     }
 
-    @Override
-    public Object getValueAndReset() {
-        return this.getMsecsUntilExpiration();
-    }
-
     public static void main(String[] args) throws Exception {
         AutoTGT at = new AutoTGT();
         Map<String, Object> conf = new java.util.HashMap();
@@ -257,6 +251,6 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetric,
     @Override
     public void registerMetrics(TopologyContext topoContext, Map<String, Object> topoConf) {
         int bucketSize = ((Number) topoConf.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)).intValue();
-        topoContext.registerMetric("TGT-TimeToExpiryMsecs", this, bucketSize);
+        topoContext.registerMetric("TGT-TimeToExpiryMsecs", () -> getMsecsUntilExpiration(), bucketSize);
     }
 }


[2/4] storm git commit: review changes

Posted by bo...@apache.org.
review changes


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/ab609514
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/ab609514
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/ab609514

Branch: refs/heads/master
Commit: ab609514c5f0581f39d6a5d78213ef38981436d4
Parents: 49ab628
Author: Aaron Gresch <ag...@yahoo-inc.com>
Authored: Tue Feb 6 16:08:48 2018 -0600
Committer: Aaron Gresch <ag...@yahoo-inc.com>
Committed: Tue Feb 6 16:08:48 2018 -0600

----------------------------------------------------------------------
 .../apache/storm/daemon/worker/WorkerState.java |  9 +++++-
 .../storm/executor/bolt/BoltExecutor.java       | 10 +++----
 .../org/apache/storm/metric/api/IMetric.java    |  8 ------
 .../storm/metric/api/IMetricsRegistrant.java    | 30 ++++++++++++++++++++
 .../storm/security/auth/kerberos/AutoTGT.java   | 15 ++++++----
 5 files changed, 51 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/ab609514/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
index e238961..1fb3be3 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/WorkerState.java
@@ -22,7 +22,14 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;

http://git-wip-us.apache.org/repos/asf/storm/blob/ab609514/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
index 01bff97..6494716 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 
-import org.apache.storm.Config;
 import org.apache.storm.Constants;
 import org.apache.storm.ICredentialsListener;
 import org.apache.storm.daemon.Task;
@@ -31,7 +30,7 @@ import org.apache.storm.daemon.metrics.BuiltinMetricsUtil;
 import org.apache.storm.daemon.worker.WorkerState;
 import org.apache.storm.executor.Executor;
 import org.apache.storm.hooks.info.BoltExecuteInfo;
-import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsRegistrant;
 import org.apache.storm.security.auth.IAutoCredentials;
 import org.apache.storm.stats.BoltExecutorStats;
 import org.apache.storm.task.IBolt;
@@ -86,11 +85,10 @@ public class BoltExecutor extends Executor {
 
                 // add any autocredential expiry metrics from the worker
                 if (workerData.getAutoCredentials() != null) {
-                    int bucketSize = ((Number) topoConf.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)).intValue();
                     for (IAutoCredentials autoCredential : workerData.getAutoCredentials()) {
-                        if (autoCredential instanceof IMetric) {
-                            IMetric metric = (IMetric)autoCredential;
-                            userContext.registerMetric(metric.getMetricName(), metric, bucketSize);
+                        if (autoCredential instanceof IMetricsRegistrant) {
+                            IMetricsRegistrant registrant = (IMetricsRegistrant)autoCredential;
+                            registrant.registerMetrics(userContext, topoConf);
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/storm/blob/ab609514/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java b/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
index a08ac49..ea5e5b1 100644
--- a/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java
@@ -29,12 +29,4 @@ public interface IMetric {
      * If this value can be reset, like with a counter, a side effect of calling this should be that the value is reset.
      */
     public Object getValueAndReset();
-
-    /**
-     * Get the metric name.
-     * @return the name of the metric
-     */
-    default String getMetricName() {
-        return this.getClass().getSimpleName();
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/ab609514/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsRegistrant.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsRegistrant.java b/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsRegistrant.java
new file mode 100644
index 0000000..cff0626
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsRegistrant.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.storm.metric.api;
+
+import java.util.Map;
+import org.apache.storm.task.TopologyContext;
+
+/**
+ * Interface to allow registration of metrics.
+ */
+public interface IMetricsRegistrant {
+    void registerMetrics(TopologyContext topoContext, Map<String, Object> topoConf);
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/ab609514/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
index 3c0a248..c0d40ad 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/AutoTGT.java
@@ -33,18 +33,21 @@ import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
 import javax.xml.bind.DatatypeConverter;
 
+import org.apache.storm.Config;
 import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.api.IMetricsRegistrant;
 import org.apache.storm.security.auth.AuthUtils;
 import org.apache.storm.security.auth.IAutoCredentials;
 import org.apache.storm.security.auth.ICredentialsRenewer;
 
+import org.apache.storm.task.TopologyContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * Automatically take a user's TGT, and push it, and renew it in Nimbus.
  */
-public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetric {
+public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetric, IMetricsRegistrant {
     private static final Logger LOG = LoggerFactory.getLogger(AutoTGT.class);
     private static final float TICKET_RENEW_WINDOW = 0.80f;
     protected static final AtomicReference<KerberosTicket> kerbTicket = new AtomicReference<>();
@@ -238,11 +241,6 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetric {
         return this.getMsecsUntilExpiration();
     }
 
-    @Override
-    public String getMetricName() {
-        return "TGT-TimeToExpiryMsecs";
-    }
-
     public static void main(String[] args) throws Exception {
         AutoTGT at = new AutoTGT();
         Map<String, Object> conf = new java.util.HashMap();
@@ -256,4 +254,9 @@ public class AutoTGT implements IAutoCredentials, ICredentialsRenewer, IMetric {
     }
 
 
+    @Override
+    public void registerMetrics(TopologyContext topoContext, Map<String, Object> topoConf) {
+        int bucketSize = ((Number) topoConf.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)).intValue();
+        topoContext.registerMetric("TGT-TimeToExpiryMsecs", this, bucketSize);
+    }
 }


[4/4] storm git commit: Merge branch 'agresch_athens' of https://github.com/agresch/storm into STORM-2935

Posted by bo...@apache.org.
Merge branch 'agresch_athens' of https://github.com/agresch/storm into STORM-2935

STORM-2935: Add metric to track when TGT expires

This closes #2548


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/5f057398
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/5f057398
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/5f057398

Branch: refs/heads/master
Commit: 5f0573983a118743e5c2eb4bb82861dd708f9014
Parents: c02f4a0 b5aca6c
Author: Robert Evans <ev...@yahoo-inc.com>
Authored: Wed Feb 7 14:39:45 2018 -0600
Committer: Robert Evans <ev...@yahoo-inc.com>
Committed: Wed Feb 7 14:39:45 2018 -0600

----------------------------------------------------------------------
 .../org/apache/storm/daemon/worker/Worker.java  | 25 ++++---
 .../apache/storm/daemon/worker/WorkerState.java | 52 +++++++++-----
 .../storm/executor/bolt/BoltExecutor.java       | 21 +++++-
 .../org/apache/storm/metric/api/IMetric.java    |  6 +-
 .../storm/metric/api/IMetricsRegistrant.java    | 30 ++++++++
 .../storm/security/INimbusCredentialPlugin.java |  7 +-
 .../storm/security/auth/kerberos/AutoTGT.java   | 73 +++++++++++++-------
 7 files changed, 151 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5f057398/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
----------------------------------------------------------------------