You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2023/01/03 18:24:42 UTC

[GitHub] [ozone] xBis7 opened a new pull request, #4140: Make OM Ratis roles available in /prom endpoint

xBis7 opened a new pull request, #4140:
URL: https://github.com/apache/ozone/pull/4140

   ## What changes were proposed in this pull request?
   
   Ratis roles are created as part of the OMMXBean class which makes them available only for `/jmx`. This information should be available for both `/jmx` and `/prom` endpoints of the OM.
   
   In order to make the `OmRatisRoles` string available for the `/prom` endpoint we can create a tag with it. We could add this tag in `OMMetrics` but tags can't be updated or changed after being created. As a workaround we created new metrics `OMHAMetrics` which we unregister and create everytime the OM list gets updated, so that the tag also gets updated. 
   
   There will be a follow up PR, with a similar approach for the SCM.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-7721
   
   ## How was this patch tested?
   
   A new test was added under `hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithData.java`. 
   
   This patch was also tested manually in docker clusters for both HA and non-HA like so
   
   * in `/hadoop-ozone/dist/target/ozone-1.4.0-SNAPSHOT/compose/ozone`
       
       ```
       ❯ docker-compose up --scale datanode=3 -d
       
       0.0.0.0:9874/jmx
       {
           "name" : "Hadoop:service=OzoneManager,name=OMHAMetrics",
           "modelerType" : "OMHAMetrics",
           "tag.OMRoles" : "STANDALONE",
           "tag.Context" : "ozone",
           "tag.Hostname" : "711d2525c257",
           "NumOfOMNodes" : 1
       },
       
       0.0.0.0:9874/prom
       # TYPE omha_metrics_num_of_om_nodes gauge
       omha_metrics_num_of_om_nodes{omroles="STANDALONE",context="ozone",hostname="711d2525c257"} 1
       ```
   
   * in `/hadoop-ozone/dist/target/ozone-1.4.0-SNAPSHOT/compose/ozone-ha` 
   
     edit `docker-compose.yaml` and forward port 9874 for om1
     ```
         ports:
           - 9874:9874
           - 9862
         hostname: om1
         command: ["ozone","om"]
     ```
     edit `docker-config` and add
     ```
     OZONE-SITE.XML_ozone.om.http-address=0.0.0.0:9874
     ```
     
     ```
     ❯ docker-compose up --scale datanode=3 -d
     
     0.0.0.0:9874/jmx
     {
         "name" : "Hadoop:service=OzoneManager,name=OMHAMetrics",
         "modelerType" : "OMHAMetrics",
         "tag.OMRoles" : " { HostName: om1 | Node-Id: om1 | Ratis-Port : 9872 | Role: FOLLOWER}  { HostName: om2 | Node-Id: om2 | Ratis-Port : 9872 | Role: LEADER}  { HostName: om3 | Node-Id: om3 | Ratis-Port : 9872 | Role: FOLLOWER} ",
         "tag.Context" : "ozone",
         "tag.Hostname" : "om1",
         "NumOfOMNodes" : 3
     },
     
     0.0.0.0:9874/prom
     # TYPE omha_metrics_num_of_om_nodes gauge
     omha_metrics_num_of_om_nodes{omroles=" { HostName: om1 | Node-Id: om1 | Ratis-Port : 9872 | Role: FOLLOWER}  { HostName: om2 | Node-Id: om2 | Ratis-Port : 9872 | Role: LEADER}  { HostName: om3 | Node-Id: om3 | Ratis-Port : 9872 | Role: FOLLOWER} ",context="ozone",hostname="om1"} 3
     ```


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1108406279


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1826,6 +1828,20 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    String leaderId = "";
+    if (isRatisEnabled) {
+      RaftPeer leader = null;
+      try {
+        leader = omRatisServer.getLeader();
+      } catch (IOException ex) {
+        LOG.error("IOException while getting the " +
+            "Ratis server leader.", ex);
+      }
+      if (Objects.nonNull(leader)) {
+        leaderId = leader.getId().toString();
+      }
+    }
+    omHAMetricsInit(leaderId);

Review Comment:
   > If `leaderId` is not set due to ratis 
   
   In that case `updatePeerList()` will never get called and `OMHAMetrics` won't be registered.
   
   > error getting leader a blank "" string is passed to `omHAMetricsInit`
   
   `OMHAMetrics` check whether the current `OMNodeId` is equal to `leaderId`. If `leaderId` is empty then this check will return false and state will be set to 0. If `leaderId` is empty, then there is no leader and setting state to follower seems reasonable to me.
   



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1433003906

   @neils-dev Double checking the `leaderId` seems unecessary. If for some reason there is no leader or there is an error and `leaderId` is empty, then the state is set to 0 pointing that the current node is a follower, which will be true.
   
   I hadn't thought about the case where ratis is disabled but I looked into it and tested it. Since, in that scenario there are no leaders or followers, `updatePeerList()` never gets called and therefore `OMHAMetrics` are not registered. Checking if ratis is enabled is redundant because if it isn't and the method gets called we will get a NPE at the very first line when calling `omRatisServer`.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1093096165


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithData.java:
##########
@@ -104,6 +106,43 @@ public void testMultipartUpload() throws Exception {
     testMultipartUploadWithOneOmNodeDown();
   }
 
+  @Test
+  public void testOMHAMetrics()
+      throws InterruptedException, TimeoutException {
+    waitForLeaderToBeReady();
+
+    OzoneManager leaderOM = getCluster().getOMLeader();
+    OzoneManager randomOM = getCluster().getOzoneManager(1);
+
+    // Get OMHAMetrics
+    OMHAMetrics omhaMetrics = randomOM.getOmhaMetrics();
+
+    if (randomOM.getOMNodeId()
+        .equals(leaderOM.getOMNodeId())) {
+      Assertions.assertEquals(1L,
+          omhaMetrics.getOmhaInfoOzoneManagerHALeaderState());
+    } else {
+      Assertions.assertEquals(0L,
+          omhaMetrics.getOmhaInfoOzoneManagerHALeaderState());
+    }
+  }

Review Comment:
   Please expand the test case (or add another one) to cover change of leadership.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithData.java:
##########
@@ -104,6 +106,43 @@ public void testMultipartUpload() throws Exception {
     testMultipartUploadWithOneOmNodeDown();
   }
 
+  @Test
+  public void testOMHAMetrics()
+      throws InterruptedException, TimeoutException {
+    waitForLeaderToBeReady();
+
+    OzoneManager leaderOM = getCluster().getOMLeader();
+    OzoneManager randomOM = getCluster().getOzoneManager(1);
+
+    // Get OMHAMetrics
+    OMHAMetrics omhaMetrics = randomOM.getOmhaMetrics();
+
+    if (randomOM.getOMNodeId()
+        .equals(leaderOM.getOMNodeId())) {
+      Assertions.assertEquals(1L,
+          omhaMetrics.getOmhaInfoOzoneManagerHALeaderState());
+    } else {
+      Assertions.assertEquals(0L,
+          omhaMetrics.getOmhaInfoOzoneManagerHALeaderState());
+    }

Review Comment:
   Instead of selecting a "random" node and having only a single assertion, the test could iterate all OMs and verify value in each.
   
   Also, please reduce duplication:
   
   ```suggestion
       int expected;
       if (randomOM.getOMNodeId()
           .equals(leaderOM.getOMNodeId())) {
         expected = 1;
       } else {
         expected = 0;
       }
       
       Assertions.assertEquals(expected,
           omhaMetrics.getOmhaInfoOzoneManagerHALeaderState());
   ```
   
   (Please feel free to replace the `if-else` with `?:`.)



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1816,6 +1818,20 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    String leaderId = "";
+    if (isRatisEnabled) {
+      RaftPeer leader = null;
+      try {
+        leader = omRatisServer.getLeader();
+      } catch (IOException ex) {
+        LOG.error("IOException while getting the " +
+            "Ratis server leader.", ex);
+      }
+      if (Objects.nonNull(leader)) {
+        leaderId += leader.getId().toString();

Review Comment:
   ```suggestion
           leaderId = leader.getId().toString();
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMHAMetrics.java:
##########
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om.ha;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * Class to maintain metrics and info related to OM HA.
+ */
+@Metrics(about = "OzoneManager HA Metrics", context = OzoneConsts.OZONE)
+public final class OMHAMetrics implements MetricsSource {
+
+  private enum OMHAMetricsInfo implements MetricsInfo {
+
+    OzoneManagerHALeaderState("Leader active state " +
+        "of OzoneManager node (1 leader, 0 follower)"),
+    NodeId("OM node Id");
+
+    private final String description;
+
+    OMHAMetricsInfo(String description) {
+      this.description = description;
+    }
+
+    @Override
+    public String description() {
+      return description;
+    }
+  }
+
+  /**
+   * Private nested class to hold
+   * the values of OMHAMetricsInfo.
+   */
+  private static final class OMHAInfo {
+
+    private long ozoneManagerHALeaderState;
+    private String nodeId;
+
+    OMHAInfo() {
+      this.ozoneManagerHALeaderState = 0L;
+      this.nodeId = "";
+    }
+
+    public long getOzoneManagerHALeaderState() {
+      return ozoneManagerHALeaderState;
+    }
+
+    public void setOzoneManagerHALeaderState(long ozoneManagerHALeaderState) {
+      this.ozoneManagerHALeaderState = ozoneManagerHALeaderState;
+    }
+
+    public String getNodeId() {
+      return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+      this.nodeId = nodeId;
+    }
+  }
+
+  public static final String SOURCE_NAME =
+      OMHAMetrics.class.getSimpleName();
+  private final OMHAInfo omhaInfo = new OMHAInfo();
+  private MetricsRegistry metricsRegistry;
+
+  private String currNodeId;
+  private String leaderId;
+
+  public OMHAMetrics(String currNodeId, String leaderId) {
+    this.currNodeId = currNodeId;
+    this.leaderId = leaderId;
+    this.metricsRegistry = new MetricsRegistry(SOURCE_NAME);
+  }
+
+  /**
+   * Create and return OMHAMetrics instance.
+   * @return OMHAMetrics
+   */
+  public static synchronized OMHAMetrics create(
+      String nodeId, String leaderId) {
+    OMHAMetrics metrics = new OMHAMetrics(nodeId, leaderId);
+    return DefaultMetricsSystem.instance()
+        .register(SOURCE_NAME, "Metrics for OM HA", metrics);
+  }
+
+  /**
+   * Unregister the metrics instance.
+   */
+  public static void unRegister() {
+    DefaultMetricsSystem.instance().unregisterSource(SOURCE_NAME);
+  }
+
+  @Override
+  public synchronized void getMetrics(MetricsCollector collector, boolean all) {
+
+    MetricsRecordBuilder recordBuilder = collector.addRecord(SOURCE_NAME);
+
+    if (currNodeId.equals(leaderId)) {
+      omhaInfo.setNodeId(currNodeId);
+      omhaInfo.setOzoneManagerHALeaderState(1);
+
+      recordBuilder
+          .tag(OMHAMetricsInfo.NodeId, currNodeId)
+          .addGauge(OMHAMetricsInfo.OzoneManagerHALeaderState, 1);
+    } else {
+      omhaInfo.setNodeId(currNodeId);
+      omhaInfo.setOzoneManagerHALeaderState(0);
+
+      recordBuilder
+          .tag(OMHAMetricsInfo.NodeId, currNodeId)
+          .addGauge(OMHAMetricsInfo.OzoneManagerHALeaderState, 0);
+    }

Review Comment:
   Please reduce duplication.  The only difference between the branches is the leader state value (0 or 1).
   
   ```suggestion
       int state = currNodeId.equals(leaderId) ? 1 : 0;
       omhaInfo.setNodeId(currNodeId);
       omhaInfo.setOzoneManagerHALeaderState(state);
   
       recordBuilder
           .tag(OMHAMetricsInfo.NodeId, currNodeId)
           .addGauge(OMHAMetricsInfo.OzoneManagerHALeaderState, state);
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1370158789

   @kerneltime Thanks for the suggestion, I included the config for OM and SCM.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1439578701

   Thanks @xBis7 for the patch, @kerneltime, @neils-dev for the review.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai merged pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai merged PR #4140:
URL: https://github.com/apache/ozone/pull/4140


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1112861655


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1826,6 +1828,20 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    String leaderId = "";
+    if (isRatisEnabled) {
+      RaftPeer leader = null;
+      try {
+        leader = omRatisServer.getLeader();
+      } catch (IOException ex) {
+        LOG.error("IOException while getting the " +
+            "Ratis server leader.", ex);
+      }
+      if (Objects.nonNull(leader)) {
+        leaderId = leader.getId().toString();
+      }
+    }
+    omHAMetricsInit(leaderId);

Review Comment:
   > Undefined in neither follower nor leader.
   
   OM Ratis Roles on the OM endpoint sets the current node's status to follower in that case. 
   
   > checking the condition here and not callling omHAMetricsInit() or similar.
   
   This seems best. I'll add a check.



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1404551502

   Thanks @xBis7.  I took a look at the prom `omha_metrics_ozone_manager_ha_leader_state` metric when it is tracked and charted on om leadership transition.  I rendered the prometheus endpoint with the prometheus web app and performed failover with _**2 om nodes**_.  On failover currently with the extra tag for "_state_" we get extra traces, in this case 4, one for each state change and gauge change, see - https://github.com/neils-dev/play/blob/main/images/failover_extra_traces.png.
   
   It would be much cleaner to keep the tags to a min and just use the gauge to reflect the leader and changes to the leader.  This can be seen when failover is rendered on prometheus with simplified tags, two traces this time, see - https://github.com/neils-dev/play/blob/main/images/failovers_just_gauge.png.  Failover is the criss-cross when rendered.
   


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1107988022


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1826,6 +1828,20 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    String leaderId = "";
+    if (isRatisEnabled) {
+      RaftPeer leader = null;
+      try {
+        leader = omRatisServer.getLeader();
+      } catch (IOException ex) {
+        LOG.error("IOException while getting the " +
+            "Ratis server leader.", ex);
+      }
+      if (Objects.nonNull(leader)) {
+        leaderId = leader.getId().toString();
+      }
+    }
+    omHAMetricsInit(leaderId);

Review Comment:
   If `leaderId` is not set due to ratis not enabled or error getting leader a blank "" string is passed to `omHAMetricsInit`.  In `omHAMetrics` iin this case what happens?  We should handle that. (see also comment in `omHAMetrics.getMetrics()`)



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMHAMetrics.java:
##########
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om.ha;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * Class to maintain metrics and info related to OM HA.
+ */
+@Metrics(about = "OzoneManager HA Metrics", context = OzoneConsts.OZONE)
+public final class OMHAMetrics implements MetricsSource {
+
+  /**
+   * Private nested class to hold the values
+   * of MetricsInfo for OMHAMetrics.
+   */
+  private static final class OMHAMetricsInfo {
+
+    private static final MetricsInfo OZONE_MANAGER_HA_LEADER_STATE =
+        Interns.info("OzoneManagerHALeaderState",
+            "Leader active state of OzoneManager node (1 leader, 0 follower)");
+
+    private static final MetricsInfo NODE_ID =
+        Interns.info("NodeId", "OM node Id");
+
+    private long ozoneManagerHALeaderState;
+    private String nodeId;
+
+    OMHAMetricsInfo() {
+      this.ozoneManagerHALeaderState = 0L;
+      this.nodeId = "";
+    }
+
+    public long getOzoneManagerHALeaderState() {
+      return ozoneManagerHALeaderState;
+    }
+
+    public void setOzoneManagerHALeaderState(long ozoneManagerHALeaderState) {
+      this.ozoneManagerHALeaderState = ozoneManagerHALeaderState;
+    }
+
+    public String getNodeId() {
+      return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+      this.nodeId = nodeId;
+    }
+  }
+
+  public static final String SOURCE_NAME =
+      OMHAMetrics.class.getSimpleName();
+  private final OMHAMetricsInfo omhaMetricsInfo = new OMHAMetricsInfo();
+  private MetricsRegistry metricsRegistry;
+
+  private String currNodeId;
+  private String leaderId;
+
+  private OMHAMetrics(String currNodeId, String leaderId) {
+    this.currNodeId = currNodeId;
+    this.leaderId = leaderId;
+    this.metricsRegistry = new MetricsRegistry(SOURCE_NAME);
+  }
+
+  /**
+   * Create and return OMHAMetrics instance.
+   * @return OMHAMetrics
+   */
+  public static OMHAMetrics create(
+      String nodeId, String leaderId) {
+    OMHAMetrics metrics = new OMHAMetrics(nodeId, leaderId);
+    return DefaultMetricsSystem.instance()
+        .register(SOURCE_NAME, "Metrics for OM HA", metrics);
+  }
+
+  /**
+   * Unregister the metrics instance.
+   */
+  public static void unRegister() {
+    DefaultMetricsSystem.instance().unregisterSource(SOURCE_NAME);
+  }
+
+  @Override
+  public synchronized void getMetrics(MetricsCollector collector, boolean all) {
+
+    MetricsRecordBuilder recordBuilder = collector.addRecord(SOURCE_NAME);
+
+    // Check current node state (1 leader, 0 follower)
+    int state = currNodeId.equals(leaderId) ? 1 : 0;

Review Comment:
   If `leaderId` is an empty string due to leader unavail at metrics registration, we should probably check here and omit setting metrics.  Just return.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithData.java:
##########
@@ -104,6 +106,83 @@ public void testMultipartUpload() throws Exception {
     testMultipartUploadWithOneOmNodeDown();
   }
 
+  @Test
+  public void testOMHAMetrics() throws InterruptedException,
+      TimeoutException, IOException {
+    waitForLeaderToBeReady();
+
+    // Get leader OM
+    OzoneManager leaderOM = getCluster().getOMLeader();
+    // Store current leader's node ID,
+    // to use it after restarting the OM
+    String leaderNodeId = leaderOM.getOMNodeId();
+    // Get a list of all OMs
+    List<OzoneManager> omList = getCluster().getOzoneManagersList();
+
+    // Check metrics for all OMs
+    checkOMHAMetricsForAllOMs(omList, leaderOM);
+
+    // Restart current leader OM
+    leaderOM.stop();
+    leaderOM.restart();
+
+    waitForLeaderToBeReady();
+
+    // Get the new leader
+    OzoneManager newLeaderOM = getCluster().getOMLeader();
+    // Get a list of all OMs again
+    omList = getCluster().getOzoneManagersList();
+
+    // New state for the old leader
+    int newState = leaderNodeId.equals(newLeaderOM.getOMNodeId()) ? 1 : 0;
+
+    // Get old leader
+    OzoneManager oldLeader = getCluster().getOzoneManager(leaderNodeId);
+    // Get old leader's metrics
+    OMHAMetrics omhaMetrics = oldLeader.getOmhaMetrics();
+
+    Assertions.assertEquals(newState,
+        omhaMetrics.getOmhaInfoOzoneManagerHALeaderState());
+
+    // Check that metrics for all OMs have been updated
+    checkOMHAMetricsForAllOMs(omList, newLeaderOM);
+  }
+
+  private void checkOMHAMetricsForAllOMs(List<OzoneManager> omList,
+                                         OzoneManager leaderOM) {

Review Comment:
   This check doesn't need the `leaderOM` `OzoneManager`.  We can just pass the `leaderNodeId` `string` and use it inside to check on line 160.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMHAMetrics.java:
##########
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om.ha;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * Class to maintain metrics and info related to OM HA.
+ */
+@Metrics(about = "OzoneManager HA Metrics", context = OzoneConsts.OZONE)
+public final class OMHAMetrics implements MetricsSource {
+
+  /**
+   * Private nested class to hold the values
+   * of MetricsInfo for OMHAMetrics.
+   */
+  private static final class OMHAMetricsInfo {
+
+    private static final MetricsInfo OZONE_MANAGER_HA_LEADER_STATE =
+        Interns.info("OzoneManagerHALeaderState",
+            "Leader active state of OzoneManager node (1 leader, 0 follower)");
+
+    private static final MetricsInfo NODE_ID =
+        Interns.info("NodeId", "OM node Id");
+
+    private long ozoneManagerHALeaderState;
+    private String nodeId;
+
+    OMHAMetricsInfo() {
+      this.ozoneManagerHALeaderState = 0L;
+      this.nodeId = "";
+    }
+
+    public long getOzoneManagerHALeaderState() {
+      return ozoneManagerHALeaderState;
+    }
+
+    public void setOzoneManagerHALeaderState(long ozoneManagerHALeaderState) {
+      this.ozoneManagerHALeaderState = ozoneManagerHALeaderState;
+    }
+
+    public String getNodeId() {
+      return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+      this.nodeId = nodeId;
+    }
+  }
+
+  public static final String SOURCE_NAME =
+      OMHAMetrics.class.getSimpleName();
+  private final OMHAMetricsInfo omhaMetricsInfo = new OMHAMetricsInfo();
+  private MetricsRegistry metricsRegistry;
+
+  private String currNodeId;
+  private String leaderId;
+
+  private OMHAMetrics(String currNodeId, String leaderId) {
+    this.currNodeId = currNodeId;
+    this.leaderId = leaderId;
+    this.metricsRegistry = new MetricsRegistry(SOURCE_NAME);
+  }
+
+  /**
+   * Create and return OMHAMetrics instance.
+   * @return OMHAMetrics
+   */
+  public static OMHAMetrics create(
+      String nodeId, String leaderId) {
+    OMHAMetrics metrics = new OMHAMetrics(nodeId, leaderId);

Review Comment:
   Thanks for the updates @xBis7.  We should check whether `leaderId` is blank or not.  If blank, log an error indicating no valid leader was given.



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1114254142


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1894,6 +1897,26 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    RaftPeer leader = null;
+    try {
+      leader = omRatisServer.getLeader();
+    } catch (IOException ex) {
+      LOG.error("IOException while getting the " +
+          "Ratis server leader.", ex);
+    }
+    if (Objects.nonNull(leader)) {
+      String leaderId = leader.getId().toString();
+
+      // If leaderId is empty, then leader is undefined
+      // and current OM is neither leader nor follower.
+      // OMHAMetrics shouldn't be registered in that case.
+      if (!Strings.isNullOrEmpty(leaderId)) {

Review Comment:
   @neils-dev @xBis7
   
   I had to update this part of the code after merging the PR due to a conflicting change that had been just merged (HDDS-6743, `getLeader()` no longer throws `IOException`).
   
   Upon closer inspection, I think these `if-else` blocks have a problem.  Please correct me if I'm wrong, but if leader is undefined then `leader` will be `null`, so the unregistration will not happen.  (This seems to have been the case even before HDDS-6743.)  If we have any leader information, its `id` cannot be `null`.
   
   I think it should be:
   
   ```java
   if (Objects.nonNull(leader)) {
     // init
   } else {
     // unregister
   }
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1370223123

   @kerneltime Thanks for looking into this. 
   
   > Why the need to format the metric as
   
   I wanted to avoid duplicating code, so I got the same string we are using in `OMMXBean.getRatisRoles()` from `OzoneManager.getRatisRoles()`.
   
   We can either split it up in new tags or have a Map and then use that for the tag. 
   
   > Would it make sense to break it up into individual values so it can be charted?
   
   We want to track the leader and if we are presenting info only for the current node, then we would have to go over all of them just to find the leader. If you want to simplify it, we could have a tag with only the info for the leader and skip all the followers.
   
   Do you have any suggestions about the format? This fix is based on this [discussion](https://github.com/apache/ozone/pull/3781#discussion_r1001614335) and #3791.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1412043165

   > I thought the goal was to avoid merging to keep the commit history linear.
   
   Each PR is squashed into a single commit, so keeping the PR's commit history linear doesn't make any difference.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
neils-dev commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1081884178


##########
hadoop-ozone/dist/src/main/compose/ozone-ha/docker-config:
##########
@@ -23,6 +23,9 @@ OZONE-SITE.XML_ozone.om.address.omservice.om2=om2
 OZONE-SITE.XML_ozone.om.address.omservice.om3=om3
 OZONE-SITE.XML_ozone.om.ratis.enable=true
 
+OZONE-SITE.XML_ozone.om.http-address=0.0.0.0:9874
+OZONE-SITE.XML_ozone.scm.http-address=0.0.0.0:9876
+

Review Comment:
   These two definitions are unnecessary.  With ports 9874 and 9876 exposed in docker containers, by default the om and scm webservers serve the ports.  Works without.



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] kerneltime commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
kerneltime commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1370172494

   Question: Why the need to format the metric as 
   ```
   { HostName: om1 | Node-Id: om1 | Ratis-Port : 9872 | Role: FOLLOWER}  { HostName: om2 | Node-Id: om2 | Ratis-Port : 9872 | Role: LEADER}  { HostName: om3 | Node-Id: om3 | Ratis-Port : 9872 | Role: FOLLOWER} 
   ``` 
   Would it make sense to break it up into individual values so it can be charted? It could be
   leader: true
   follower: false
   ratis-port:
   and so on...
   I think Prometheus metrics have a specific scheme to it. 


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] kerneltime commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
kerneltime commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1370142027

   Do you want to include `docker-compose.yaml` and `docker-config` changes? I think being able to read the metrics would be useful in general. 


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1408295958

   Hey @DaveTeng0, seems unrelated. Check [here](https://github.com/xBis7/ozone/actions/runs/4017143290), I had a green build on the workflow of my fork. A lot of tests are flaky but I don't have write privileges to rerun the failed ones on the PR workflow.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1412479337

   Thanks @xBis7 , @adoroszlai, I'm going to take another look.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1114338731


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1894,6 +1897,26 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    RaftPeer leader = null;
+    try {
+      leader = omRatisServer.getLeader();
+    } catch (IOException ex) {
+      LOG.error("IOException while getting the " +
+          "Ratis server leader.", ex);
+    }
+    if (Objects.nonNull(leader)) {
+      String leaderId = leader.getId().toString();
+
+      // If leaderId is empty, then leader is undefined
+      // and current OM is neither leader nor follower.
+      // OMHAMetrics shouldn't be registered in that case.
+      if (!Strings.isNullOrEmpty(leaderId)) {

Review Comment:
   @adoroszlai You are right. This was missed because the code was 
   
   ```java
   String leaderId = "";
   try{
     leader = 
   } catch() {
   
   }
   
   if (Objects.nonNull(leader)) {
   
   }
   
   // leaderId could be deliberately left empty down here due to failure to get the leader
   // after refactoring `String leaderId = "";` was removed.
   ```
   
   > If we have any leader information, its id cannot be null.
   
   I didn't know that.
   
   How can we handle this now since the code has been merged?



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1114369999


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1894,6 +1897,26 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    RaftPeer leader = null;
+    try {
+      leader = omRatisServer.getLeader();
+    } catch (IOException ex) {
+      LOG.error("IOException while getting the " +
+          "Ratis server leader.", ex);
+    }
+    if (Objects.nonNull(leader)) {
+      String leaderId = leader.getId().toString();
+
+      // If leaderId is empty, then leader is undefined
+      // and current OM is neither leader nor follower.
+      // OMHAMetrics shouldn't be registered in that case.
+      if (!Strings.isNullOrEmpty(leaderId)) {

Review Comment:
   @adoroszlai Thanks! I'll create a patch shortly for [HDDS-8009](https://issues.apache.org/jira/browse/HDDS-8009).



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1082339057


##########
hadoop-ozone/dist/src/main/compose/ozone-ha/docker-config:
##########
@@ -23,6 +23,9 @@ OZONE-SITE.XML_ozone.om.address.omservice.om2=om2
 OZONE-SITE.XML_ozone.om.address.omservice.om3=om3
 OZONE-SITE.XML_ozone.om.ratis.enable=true
 
+OZONE-SITE.XML_ozone.om.http-address=0.0.0.0:9874
+OZONE-SITE.XML_ozone.scm.http-address=0.0.0.0:9876
+

Review Comment:
   You are right, I'll remove them.



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1412167142

   @adoroszlai I've updated the patch, with the changes you requested. Can you please take another look?


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1433104062

   @neils-dev I've added some unit tests, so that we can check the case where we provide an empty `leaderId`.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1112479321


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsSnapshot.java:
##########
@@ -63,7 +63,7 @@ public class TestOzoneFsSnapshot {
   private static OzoneManager ozoneManager;
   private static OzoneFsShell shell;
   private static final String VOLUME =
-      "vol-" + RandomStringUtils.randomNumeric(5);;
+      "vol-" + RandomStringUtils.randomNumeric(5);

Review Comment:
   This file was included in the latest commit.  Looks like unintended.  Please revert this addition.



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1438252778

   @neils-dev Thanks for the reviews, I've updated the patch. Let me know how it looks.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
neils-dev commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1081818641


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithData.java:
##########
@@ -104,6 +105,30 @@ public void testMultipartUpload() throws Exception {
     testMultipartUploadWithOneOmNodeDown();
   }
 
+  @Test
+  public void testOMHAMetrics() throws InterruptedException {
+    Thread.sleep(2000);

Review Comment:
   Having a sleep in a test can be a cause of 'flaky' tests depending on the load of system at time of test.  Can we change this to be a sleep conditional and use a `genericTestUtils.waitFor(() -> {...}, checkEveryMillis, waitForMillis)`?



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1082399805


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerHAWithData.java:
##########
@@ -104,6 +105,30 @@ public void testMultipartUpload() throws Exception {
     testMultipartUploadWithOneOmNodeDown();
   }
 
+  @Test
+  public void testOMHAMetrics() throws InterruptedException {
+    Thread.sleep(2000);

Review Comment:
   Thanks for pointing that out. I fixed it!



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1405226678

   @neils-dev I've tested it locally and I can see what you are referring to. I will remove `state` tag from `OMHAMetrics. Thanks!


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1440068483

   > [HDDS-8009](https://issues.apache.org/jira/browse/HDDS-8009)
   
   @adoroszlai Thanks! I'll create a patch shortly.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1112477407


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1826,6 +1828,20 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    String leaderId = "";
+    if (isRatisEnabled) {
+      RaftPeer leader = null;
+      try {
+        leader = omRatisServer.getLeader();
+      } catch (IOException ex) {
+        LOG.error("IOException while getting the " +
+            "Ratis server leader.", ex);
+      }
+      if (Objects.nonNull(leader)) {
+        leaderId = leader.getId().toString();
+      }
+    }
+    omHAMetricsInit(leaderId);

Review Comment:
   > If leaderId is empty, then there is no leader and setting state to follower seems reasonable to me.
   
   Thanks @xBis7 for the comments on this.  In this case, the leader is undefined.  Undefined in neither follower nor leader.  We should indicate this as such in the `OMHAMetrics `- handling this empty string `leaderId` in the `OMHAMetrics.create()`, checking the condition here and _not_ callling `omHAMetricsInit()` or similar.



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1437783402

   > Checking if ratis is enabled is redundant because if it isn't and the method gets called we will get a NPE at the very first line when calling omRatisServer. I'll remove the check.
   
   Thanks for updating that.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1398265160

   @neils-dev Thanks for the review! I have addressed your comments.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1412000500

   @xBis7 Please try to avoid force-push when updating the PR.  Here are some great articles that explain why:
   
   https://developers.mattermost.com/blog/submitting-great-prs/#4-avoid-force-pushing
   https://www.freecodecamp.org/news/optimize-pull-requests-for-reviewer-happiness#request-a-review


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1412402344

   > If I understand correctly, @neils-dev's comment was also addressed previously in https://github.com/apache/ozone/commit/e836659ea975c95f02e57ab90deb801e9c0f7eeb. 
   
   @adoroszlai Yes, it has been addressed.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1413616145

   @neils-dev I've addressed your comments. Can you please take another look?


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1113385807


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1894,6 +1897,23 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    RaftPeer leader = null;
+    try {
+      leader = omRatisServer.getLeader();
+    } catch (IOException ex) {
+      LOG.error("IOException while getting the " +
+          "Ratis server leader.", ex);
+    }
+    if (Objects.nonNull(leader)) {
+      String leaderId = leader.getId().toString();
+
+      // If leaderId is empty, then leader is undefined
+      // and current OM is neither leader nor follower.
+      // OMHAMetrics shouldn't be registered in that case.
+      if (!Strings.isNullOrEmpty(leaderId)) {

Review Comment:
   Thanks @xBis7 .  Check looks good.  If the `leaderId `isEmpty() also we should also _unregister_ the metric so we do not have stale metrics.



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1438895865

   @neils-dev I've updated the patch to address the latest comment.


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1114659218


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1894,6 +1897,26 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    RaftPeer leader = null;
+    try {
+      leader = omRatisServer.getLeader();
+    } catch (IOException ex) {
+      LOG.error("IOException while getting the " +
+          "Ratis server leader.", ex);
+    }
+    if (Objects.nonNull(leader)) {
+      String leaderId = leader.getId().toString();
+
+      // If leaderId is empty, then leader is undefined
+      // and current OM is neither leader nor follower.
+      // OMHAMetrics shouldn't be registered in that case.
+      if (!Strings.isNullOrEmpty(leaderId)) {

Review Comment:
   Thanks @adoroszlai for merging and uncovering this change.  With `getLeader() `no longer throwing an exception, we can cleanup the null check and handling of that condition.



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] DaveTeng0 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "DaveTeng0 (via GitHub)" <gi...@apache.org>.
DaveTeng0 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1427568170

   Once @neils-dev takes a final look, this PR would be ready to be merged!


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1381997264

   @kerneltime I updated the patch with the changes you requested. Now `OMHAMetrics` hold info only for the component registering them. All the info is added as tags instead of a concatenated string and also there is a gauge(1 for leader, 0 for follower) so that the metrics can be charted like you suggested. Can you take another look?


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] DaveTeng0 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "DaveTeng0 (via GitHub)" <gi...@apache.org>.
DaveTeng0 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1408221460

   Hey @xBis7 ! Seemed like some test was failing, please help take a look! Thanks~


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] xBis7 commented on pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "xBis7 (via GitHub)" <gi...@apache.org>.
xBis7 commented on PR #4140:
URL: https://github.com/apache/ozone/pull/4140#issuecomment-1412033590

   @adoroszlai Thanks for the review, I'll make all the changes.
   
   > Please try to avoid force-push when updating the PR. Here are some great articles that explain why:
   
   I didn't know that. I thought the goal was to avoid merging to keep the commit history linear. Thanks for sharing these articles. I'll keep their points in mind next time!


-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1094141293


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMHAMetrics.java:
##########
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om.ha;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * Class to maintain metrics and info related to OM HA.
+ */
+@Metrics(about = "OzoneManager HA Metrics", context = OzoneConsts.OZONE)
+public final class OMHAMetrics implements MetricsSource {
+
+  private enum OMHAMetricsInfo implements MetricsInfo {
+
+    OzoneManagerHALeaderState("Leader active state " +
+        "of OzoneManager node (1 leader, 0 follower)"),
+    NodeId("OM node Id");
+
+    private final String description;
+
+    OMHAMetricsInfo(String description) {
+      this.description = description;
+    }
+
+    @Override
+    public String description() {
+      return description;
+    }
+  }
+
+  /**
+   * Private nested class to hold
+   * the values of OMHAMetricsInfo.
+   */
+  private static final class OMHAInfo {
+
+    private long ozoneManagerHALeaderState;
+    private String nodeId;
+
+    OMHAInfo() {
+      this.ozoneManagerHALeaderState = 0L;
+      this.nodeId = "";
+    }
+
+    public long getOzoneManagerHALeaderState() {
+      return ozoneManagerHALeaderState;
+    }
+
+    public void setOzoneManagerHALeaderState(long ozoneManagerHALeaderState) {
+      this.ozoneManagerHALeaderState = ozoneManagerHALeaderState;
+    }
+
+    public String getNodeId() {
+      return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+      this.nodeId = nodeId;
+    }
+  }
+
+  public static final String SOURCE_NAME =
+      OMHAMetrics.class.getSimpleName();
+  private final OMHAInfo omhaInfo = new OMHAInfo();
+  private MetricsRegistry metricsRegistry;
+
+  private String currNodeId;
+  private String leaderId;
+
+  public OMHAMetrics(String currNodeId, String leaderId) {
+    this.currNodeId = currNodeId;
+    this.leaderId = leaderId;
+    this.metricsRegistry = new MetricsRegistry(SOURCE_NAME);
+  }
+
+  /**
+   * Create and return OMHAMetrics instance.
+   * @return OMHAMetrics
+   */
+  public static synchronized OMHAMetrics create(

Review Comment:
   Does this need to be _synchonized_?  An intrinsic lock?  There should be a single access to create and register with the `MetricsSystem`, then it will pull the metrics periodically.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMHAMetrics.java:
##########
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om.ha;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * Class to maintain metrics and info related to OM HA.
+ */
+@Metrics(about = "OzoneManager HA Metrics", context = OzoneConsts.OZONE)
+public final class OMHAMetrics implements MetricsSource {
+
+  private enum OMHAMetricsInfo implements MetricsInfo {
+
+    OzoneManagerHALeaderState("Leader active state " +
+        "of OzoneManager node (1 leader, 0 follower)"),
+    NodeId("OM node Id");
+
+    private final String description;
+
+    OMHAMetricsInfo(String description) {
+      this.description = description;
+    }
+
+    @Override
+    public String description() {
+      return description;
+    }
+  }
+
+  /**
+   * Private nested class to hold
+   * the values of OMHAMetricsInfo.
+   */
+  private static final class OMHAInfo {
+
+    private long ozoneManagerHALeaderState;
+    private String nodeId;
+
+    OMHAInfo() {
+      this.ozoneManagerHALeaderState = 0L;
+      this.nodeId = "";
+    }
+
+    public long getOzoneManagerHALeaderState() {
+      return ozoneManagerHALeaderState;
+    }
+
+    public void setOzoneManagerHALeaderState(long ozoneManagerHALeaderState) {
+      this.ozoneManagerHALeaderState = ozoneManagerHALeaderState;
+    }
+
+    public String getNodeId() {
+      return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+      this.nodeId = nodeId;
+    }
+  }
+
+  public static final String SOURCE_NAME =
+      OMHAMetrics.class.getSimpleName();
+  private final OMHAInfo omhaInfo = new OMHAInfo();
+  private MetricsRegistry metricsRegistry;
+
+  private String currNodeId;
+  private String leaderId;
+
+  public OMHAMetrics(String currNodeId, String leaderId) {

Review Comment:
   This should be private.  The constructor is only accessed by this class.  Users create an instance through the static `create` method,  OMHAMetrics.create, 
   `public static synchronized OMHAMetrics create`



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ha/OMHAMetrics.java:
##########
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om.ha;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * Class to maintain metrics and info related to OM HA.
+ */
+@Metrics(about = "OzoneManager HA Metrics", context = OzoneConsts.OZONE)
+public final class OMHAMetrics implements MetricsSource {
+
+  private enum OMHAMetricsInfo implements MetricsInfo {

Review Comment:
   Thanks @xBis7 for the recent changes.  For the `MetricsInfo` used in the `OMHAMetrics` tag and gauge, we should re-use the Interns.info static method just as the other metrics instances do,
   https://github.com/apache/hadoop/blob/03cfc852791c14fad39db4e5b14104a276c08e59/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/Interns.java#L117.
   
   In our case we can declare 2 static `MetricsInfo` members in the static nested class, `OMHAInfo,` one for the `OzoneManagerHALeaderState`, the other for the `NodeID`, ie:
   
   ```
   OMHAInfo {
   ...
   private static final MetricsInfo OZONEMANAGER_HA_LEADER_STATE = Interns.info(
           "OzoneManagerHALeaderState",
           <desc>);
   
   }
   ...
   ```



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1112477407


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1826,6 +1828,20 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    String leaderId = "";
+    if (isRatisEnabled) {
+      RaftPeer leader = null;
+      try {
+        leader = omRatisServer.getLeader();
+      } catch (IOException ex) {
+        LOG.error("IOException while getting the " +
+            "Ratis server leader.", ex);
+      }
+      if (Objects.nonNull(leader)) {
+        leaderId = leader.getId().toString();
+      }
+    }
+    omHAMetricsInit(leaderId);

Review Comment:
   > If leaderId is empty, then there is no leader and setting state to follower seems reasonable to me.
   
   Thanks @xBis7 for the comments on this.  In this case, the leader is undefined.  Undefined in neither follower nor leader.  We should indicate this as such in the `OMHAMetrics `- handling this empty string `leaderId` in the `OMHAMetrics.create()`, or checking the condition here and _not_ callling `omHAMetricsInit()` or similar.



-- 
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: issues-unsubscribe@ozone.apache.org

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


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


[GitHub] [ozone] adoroszlai commented on a diff in pull request #4140: HDDS-7721. Make OM Ratis roles available in /prom endpoint

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on code in PR #4140:
URL: https://github.com/apache/ozone/pull/4140#discussion_r1114354267


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -1894,6 +1897,26 @@ public void updatePeerList(List<String> newPeers) {
         }
       }
     }
+    RaftPeer leader = null;
+    try {
+      leader = omRatisServer.getLeader();
+    } catch (IOException ex) {
+      LOG.error("IOException while getting the " +
+          "Ratis server leader.", ex);
+    }
+    if (Objects.nonNull(leader)) {
+      String leaderId = leader.getId().toString();
+
+      // If leaderId is empty, then leader is undefined
+      // and current OM is neither leader nor follower.
+      // OMHAMetrics shouldn't be registered in that case.
+      if (!Strings.isNullOrEmpty(leaderId)) {

Review Comment:
   Thanks @xBis7 for checking.
   
   > > If we have any leader information, its id cannot be null.
   > 
   > I didn't know that.
   
   https://github.com/apache/ratis/blob/27f5a59cc697ce51f1c5ab6d7b6c63eb58390b79/ratis-common/src/main/java/org/apache/ratis/protocol/RaftPeer.java#L175
   
   > How can we handle this now since the code has been merged?
   
   HDDS-8009



-- 
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: issues-unsubscribe@ozone.apache.org

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


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