You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2022/04/14 22:12:24 UTC

[GitHub] [hadoop] goiri commented on a diff in pull request #4092: HADOOP-18167. Add metrics to track delegation token secret manager op…

goiri commented on code in PR #4092:
URL: https://github.com/apache/hadoop/pull/4092#discussion_r850843618


##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java:
##########
@@ -825,4 +859,68 @@ protected void syncTokenOwnerStats() {
       addTokenForOwnerStats(id);
     }
   }
+
+  /**
+   * DelegationTokenSecretManagerMetrics tracks token management operations
+   * and publishes them through the metrics interfaces.
+   */
+  @Metrics(about="Delegation token secret manager metrics", context="token")
+  static class DelegationTokenSecretManagerMetrics implements IOStatisticsSource {
+    private static final Logger LOG = LoggerFactory.getLogger(DelegationTokenSecretManagerMetrics.class);
+
+    final static String STORE_TOKEN_STAT = "storeToken";
+    final static String UPDATE_TOKEN_STAT = "updateToken";
+    final static String REMOVE_TOKEN_STAT = "removeToken";
+    final static String TOKEN_FAILURE_STAT = "tokenFailure";
+
+    final MetricsRegistry registry;
+    final IOStatisticsStore ioStatistics;
+
+    @Metric("Rate of storage of delegation tokens and latency (milliseconds)")
+    MutableRate storeToken;
+    @Metric("Rate of update of delegation tokens and latency (milliseconds)")
+    MutableRate updateToken;
+    @Metric("Rate of removal of delegation tokens and latency (milliseconds)")
+    MutableRate removeToken;
+    @Metric("Counter of delegation tokens operation failures")
+    MutableCounterLong tokenFailure;
+
+    static DelegationTokenSecretManagerMetrics create() {
+      return DefaultMetricsSystem.instance().register(new DelegationTokenSecretManagerMetrics());
+    }
+
+    public DelegationTokenSecretManagerMetrics() {
+      ioStatistics = iostatisticsStore()
+          .withDurationTracking(STORE_TOKEN_STAT, UPDATE_TOKEN_STAT, REMOVE_TOKEN_STAT)
+          .withCounters(TOKEN_FAILURE_STAT)
+          .build();
+      registry = new MetricsRegistry("DelegationTokenSecretManagerMetrics");
+      LOG.debug("Initialized {}", registry);
+    }
+
+    public void addStoreToken(long value) {

Review Comment:
   addTimeStoreToken()



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java:
##########
@@ -825,4 +859,68 @@ protected void syncTokenOwnerStats() {
       addTokenForOwnerStats(id);
     }
   }
+
+  /**
+   * DelegationTokenSecretManagerMetrics tracks token management operations
+   * and publishes them through the metrics interfaces.
+   */
+  @Metrics(about="Delegation token secret manager metrics", context="token")
+  static class DelegationTokenSecretManagerMetrics implements IOStatisticsSource {
+    private static final Logger LOG = LoggerFactory.getLogger(DelegationTokenSecretManagerMetrics.class);
+
+    final static String STORE_TOKEN_STAT = "storeToken";
+    final static String UPDATE_TOKEN_STAT = "updateToken";
+    final static String REMOVE_TOKEN_STAT = "removeToken";
+    final static String TOKEN_FAILURE_STAT = "tokenFailure";
+
+    final MetricsRegistry registry;
+    final IOStatisticsStore ioStatistics;
+
+    @Metric("Rate of storage of delegation tokens and latency (milliseconds)")
+    MutableRate storeToken;
+    @Metric("Rate of update of delegation tokens and latency (milliseconds)")
+    MutableRate updateToken;
+    @Metric("Rate of removal of delegation tokens and latency (milliseconds)")
+    MutableRate removeToken;
+    @Metric("Counter of delegation tokens operation failures")
+    MutableCounterLong tokenFailure;
+
+    static DelegationTokenSecretManagerMetrics create() {
+      return DefaultMetricsSystem.instance().register(new DelegationTokenSecretManagerMetrics());
+    }
+
+    public DelegationTokenSecretManagerMetrics() {
+      ioStatistics = iostatisticsStore()
+          .withDurationTracking(STORE_TOKEN_STAT, UPDATE_TOKEN_STAT, REMOVE_TOKEN_STAT)
+          .withCounters(TOKEN_FAILURE_STAT)
+          .build();
+      registry = new MetricsRegistry("DelegationTokenSecretManagerMetrics");
+      LOG.debug("Initialized {}", registry);
+    }
+
+    public void addStoreToken(long value) {
+      storeToken.add(value);
+      ioStatistics.addTimedOperation(STORE_TOKEN_STAT, value);
+    }
+
+    public void addUpdateToken(long value) {

Review Comment:
   addTimeUpdateToken() and same for the rest.



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java:
##########
@@ -825,4 +859,68 @@ protected void syncTokenOwnerStats() {
       addTokenForOwnerStats(id);
     }
   }
+
+  /**
+   * DelegationTokenSecretManagerMetrics tracks token management operations
+   * and publishes them through the metrics interfaces.
+   */
+  @Metrics(about="Delegation token secret manager metrics", context="token")
+  static class DelegationTokenSecretManagerMetrics implements IOStatisticsSource {
+    private static final Logger LOG = LoggerFactory.getLogger(DelegationTokenSecretManagerMetrics.class);
+
+    final static String STORE_TOKEN_STAT = "storeToken";
+    final static String UPDATE_TOKEN_STAT = "updateToken";
+    final static String REMOVE_TOKEN_STAT = "removeToken";
+    final static String TOKEN_FAILURE_STAT = "tokenFailure";
+
+    final MetricsRegistry registry;
+    final IOStatisticsStore ioStatistics;
+
+    @Metric("Rate of storage of delegation tokens and latency (milliseconds)")
+    MutableRate storeToken;
+    @Metric("Rate of update of delegation tokens and latency (milliseconds)")
+    MutableRate updateToken;
+    @Metric("Rate of removal of delegation tokens and latency (milliseconds)")
+    MutableRate removeToken;
+    @Metric("Counter of delegation tokens operation failures")
+    MutableCounterLong tokenFailure;
+
+    static DelegationTokenSecretManagerMetrics create() {
+      return DefaultMetricsSystem.instance().register(new DelegationTokenSecretManagerMetrics());
+    }
+
+    public DelegationTokenSecretManagerMetrics() {
+      ioStatistics = iostatisticsStore()
+          .withDurationTracking(STORE_TOKEN_STAT, UPDATE_TOKEN_STAT, REMOVE_TOKEN_STAT)
+          .withCounters(TOKEN_FAILURE_STAT)
+          .build();
+      registry = new MetricsRegistry("DelegationTokenSecretManagerMetrics");
+      LOG.debug("Initialized {}", registry);
+    }
+
+    public void addStoreToken(long value) {

Review Comment:
   Make the argument final too.



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java:
##########
@@ -579,4 +615,65 @@ public void testEmptyToken() throws IOException {
     assertEquals(token1, token2);
     assertEquals(token1.encodeToUrlString(), token2.encodeToUrlString());
   }
+
+  @Test
+  public void testDelegationTokenSecretManagerMetrics() throws Exception {
+    TestDelegationTokenSecretManager dtSecretManager =
+        new TestDelegationTokenSecretManager(24*60*60*1000,
+            10*1000,1*1000,3600000);
+    try {
+      dtSecretManager.startThreads();
+
+      Assert.assertEquals(0, dtSecretManager.metrics.storeToken.lastStat().numSamples());
+      final Token<TestDelegationTokenIdentifier> token =
+          generateDelegationToken(dtSecretManager, "SomeUser", "JobTracker");
+      Assert.assertEquals(1, dtSecretManager.metrics.storeToken.lastStat().numSamples());
+
+      Assert.assertEquals(0, dtSecretManager.metrics.updateToken.lastStat().numSamples());
+      dtSecretManager.renewToken(token, "JobTracker");
+      Assert.assertEquals(1, dtSecretManager.metrics.updateToken.lastStat().numSamples());
+
+      Assert.assertEquals(0, dtSecretManager.metrics.removeToken.lastStat().numSamples());
+      dtSecretManager.cancelToken(token, "JobTracker");
+      Assert.assertEquals(1, dtSecretManager.metrics.removeToken.lastStat().numSamples());
+    } finally {
+      dtSecretManager.stopThreads();
+    }
+  }
+
+  @Test
+  public void testDelegationTokenSecretManagerMetricsFailures() throws Exception {
+    TestFailureDelegationTokenSecretManager dtSecretManager = new TestFailureDelegationTokenSecretManager();
+
+    try {
+      dtSecretManager.startThreads();
+
+      final Token<TestDelegationTokenIdentifier> token =
+          generateDelegationToken(dtSecretManager, "SomeUser", "JobTracker");
+
+      dtSecretManager.setThrowError(true);
+
+      Assert.assertEquals(0, dtSecretManager.metrics.tokenFailure.value());
+      generateDelegationToken(dtSecretManager, "SomeUser", "JobTracker");
+      Assert.assertEquals(1, dtSecretManager.metrics.tokenFailure.value());
+
+      try {
+        dtSecretManager.renewToken(token, "JobTracker");
+        Assert.fail("Expected exception");
+      } catch (Exception ex) {
+        // Expected exception
+      }
+      Assert.assertEquals(2, dtSecretManager.metrics.tokenFailure.value());
+
+      try {
+        dtSecretManager.cancelToken(token, "JobTracker");
+        Assert.fail("Expected exception");

Review Comment:
   use LamdaTestUtils#intercept



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java:
##########
@@ -50,6 +59,9 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.*;

Review Comment:
   Better to expand this or to use the full name.



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java:
##########
@@ -579,4 +615,65 @@ public void testEmptyToken() throws IOException {
     assertEquals(token1, token2);
     assertEquals(token1.encodeToUrlString(), token2.encodeToUrlString());
   }
+
+  @Test
+  public void testDelegationTokenSecretManagerMetrics() throws Exception {
+    TestDelegationTokenSecretManager dtSecretManager =
+        new TestDelegationTokenSecretManager(24*60*60*1000,
+            10*1000,1*1000,3600000);

Review Comment:
   spaces after the commas



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java:
##########
@@ -825,4 +859,68 @@ protected void syncTokenOwnerStats() {
       addTokenForOwnerStats(id);
     }
   }
+
+  /**
+   * DelegationTokenSecretManagerMetrics tracks token management operations
+   * and publishes them through the metrics interfaces.
+   */
+  @Metrics(about="Delegation token secret manager metrics", context="token")
+  static class DelegationTokenSecretManagerMetrics implements IOStatisticsSource {
+    private static final Logger LOG = LoggerFactory.getLogger(DelegationTokenSecretManagerMetrics.class);
+
+    final static String STORE_TOKEN_STAT = "storeToken";
+    final static String UPDATE_TOKEN_STAT = "updateToken";
+    final static String REMOVE_TOKEN_STAT = "removeToken";
+    final static String TOKEN_FAILURE_STAT = "tokenFailure";
+
+    final MetricsRegistry registry;
+    final IOStatisticsStore ioStatistics;
+
+    @Metric("Rate of storage of delegation tokens and latency (milliseconds)")
+    MutableRate storeToken;
+    @Metric("Rate of update of delegation tokens and latency (milliseconds)")
+    MutableRate updateToken;
+    @Metric("Rate of removal of delegation tokens and latency (milliseconds)")
+    MutableRate removeToken;
+    @Metric("Counter of delegation tokens operation failures")
+    MutableCounterLong tokenFailure;
+
+    static DelegationTokenSecretManagerMetrics create() {
+      return DefaultMetricsSystem.instance().register(new DelegationTokenSecretManagerMetrics());
+    }
+
+    public DelegationTokenSecretManagerMetrics() {
+      ioStatistics = iostatisticsStore()
+          .withDurationTracking(STORE_TOKEN_STAT, UPDATE_TOKEN_STAT, REMOVE_TOKEN_STAT)
+          .withCounters(TOKEN_FAILURE_STAT)
+          .build();
+      registry = new MetricsRegistry("DelegationTokenSecretManagerMetrics");
+      LOG.debug("Initialized {}", registry);
+    }
+
+    public void addStoreToken(long value) {

Review Comment:
   And probably time and the unit instead of plain "value"



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java:
##########
@@ -579,4 +615,65 @@ public void testEmptyToken() throws IOException {
     assertEquals(token1, token2);
     assertEquals(token1.encodeToUrlString(), token2.encodeToUrlString());
   }
+
+  @Test
+  public void testDelegationTokenSecretManagerMetrics() throws Exception {
+    TestDelegationTokenSecretManager dtSecretManager =
+        new TestDelegationTokenSecretManager(24*60*60*1000,
+            10*1000,1*1000,3600000);

Review Comment:
   Extend the 3600000 too.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org