You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2019/03/30 01:20:21 UTC

[GitHub] [incubator-pinot] sunithabeeram commented on a change in pull request #4037: Add more logs to ServiceStatus

sunithabeeram commented on a change in pull request #4037: Add more logs to ServiceStatus
URL: https://github.com/apache/incubator-pinot/pull/4037#discussion_r270607485
 
 

 ##########
 File path: pinot-common/src/main/java/org/apache/pinot/common/utils/ServiceStatus.java
 ##########
 @@ -227,70 +230,98 @@ private boolean isDone() {
     @Override
     public synchronized Status getServiceStatus() {
 
+      String descriptionSuffix = "";
       while (!isDone()) {
         String resourceName;
         if (_resourceIterator == null || !_resourceIterator.hasNext()) {
           _resourceIterator = _resourcesToMonitor.iterator();
         }
         resourceName = _resourceIterator.next();
-        IdealState idealState = getResourceIdealState(resourceName);
+        descriptionSuffix = String
+            .format("waitingFor=%s, resource=%s, numResourcesLeft=%d, numTotalResources=%d, minStartCount=%d,", getMatchName(),
+                resourceName, _resourcesToMonitor.size(), _numTotalResourcesToMonitor, _minResourcesStartCount);
 
-        // If the resource has been removed or disabled, ignore it
-        if (idealState == null || !idealState.isEnabled()) {
+        Pair<Status, String> statusPair = evaluateResourceStatus(resourceName, descriptionSuffix);
+
+        // Resource is done starting up, remove it from the set
+        _statusDescription = statusPair.getSecond();
+        Status status = statusPair.getFirst();
+        if (status == Status.GOOD) {
           _resourceIterator.remove();
-          continue;
+        } else {
+          return status;
         }
+      }
 
-        String descriptionSuffix = String
-            .format("waitingFor=%s, resource=%s, numResourcesLeft=%d, numTotalResources=%d", getMatchName(),
-                resourceName, _resourcesToMonitor.size(), _numTotalResourcesToMonitor);
-        T helixState = getState(resourceName);
-        if (helixState == null) {
-          _statusDescription = "Helix state does not exist: " + descriptionSuffix;
-          return Status.STARTING;
+      if (_resourcesToMonitor.isEmpty()) {
+        _statusDescription = STATUS_DESCRIPTION_NONE;
+        LOGGER.info("Instance {} has finished starting up", _instanceName);
+      } else {
+        _statusDescription = String.format("waitingFor=%s, numResourcesLeft=%d, numTotalResources=%d, minStartCount=%d,"
+                + " resourceList=%s", getMatchName(), _resourcesToMonitor.size(), _numTotalResourcesToMonitor,
+            _minResourcesStartCount, getResourceListAsString());
+        LOGGER.info("Instance {} returning GOOD because {}", _instanceName, _statusDescription);
+
+        int logCount = MAX_RESOURCE_NAMES_TO_LOG;
+        for (String resource : _resourcesToMonitor) {
+          Pair<Status, String> statusPair = evaluateResourceStatus(resource, "");
+          LOGGER.info("Resource: {}, StatusDescription: {}", resource, statusPair.getSecond());
+          if (--logCount <= 0) {
+            break;
+          }
         }
+      }
 
-        // Check that all partitions that are supposed to be in any state other than OFFLINE have the same status in the
-        // external view or went to ERROR state (which means that we tried to load the segments/resources but failed for
-        // some reason)
-        Map<String, String> partitionStateMap = getPartitionStateMap(helixState);
-        for (String partitionName : idealState.getPartitionSet()) {
-          String idealStateStatus = idealState.getInstanceStateMap(partitionName).get(_instanceName);
+      return Status.GOOD;
+    }
 
-          // Skip this partition if it is not assigned to this instance or if the instance should be offline
-          if (idealStateStatus == null || "OFFLINE".equals(idealStateStatus)) {
-            continue;
-          }
+    private Pair<Status, String> evaluateResourceStatus(String resourceName, String descriptionSuffix) {
+      IdealState idealState = getResourceIdealState(resourceName);
+      // If the resource has been removed or disabled, ignore it
+      if (idealState == null || !idealState.isEnabled()) {
+        return new Pair(Status.GOOD, STATUS_DESCRIPTION_NONE);
+      }
 
-          // If the instance state is not ERROR and is not the same as what's expected from the ideal state, then it
-          // hasn't finished starting up
-          String currentStateStatus = partitionStateMap.get(partitionName);
-          if (!idealStateStatus.equals(currentStateStatus)) {
-            if ("ERROR".equals(currentStateStatus)) {
-              LOGGER.error(String.format("Resource: %s, partition: %s is in ERROR state", resourceName, partitionName));
-            } else {
-              _statusDescription = String
-                  .format("partition=%s, expected=%s, found=%s, %s", partitionName,
-                      idealStateStatus, currentStateStatus, descriptionSuffix);
-              return Status.STARTING;
-            }
-          }
+      T helixState = getState(resourceName);
+      if (helixState == null) {
+        String description = String.format("Helix state does not exist: %s", descriptionSuffix);
+        return new Pair(Status.STARTING, description);
+      }
+
+      // Check that all partitions that are supposed to be in any state other than OFFLINE have the same status in the
+      // external view or went to ERROR state (which means that we tried to load the segments/resources but failed for
+      // some reason)
+      Map<String, String> partitionStateMap = getPartitionStateMap(helixState);
+      for (String partitionName : idealState.getPartitionSet()) {
+        String idealStateStatus = idealState.getInstanceStateMap(partitionName).get(_instanceName);
+
+        // Skip this partition if it is not assigned to this instance or if the instance should be offline
+        if (idealStateStatus == null || "OFFLINE".equals(idealStateStatus)) {
+          continue;
         }
 
-        // Resource is done starting up, remove it from the set
-        _resourceIterator.remove();
+        // If the instance state is not ERROR and is not the same as what's expected from the ideal state, then it
+        // hasn't finished starting up
+        String currentStateStatus = partitionStateMap.get(partitionName);
+        if (!idealStateStatus.equals(currentStateStatus)) {
+          if ("ERROR".equals(currentStateStatus)) {
+            LOGGER.error(String.format("Resource: %s, partition: %s is in ERROR state", resourceName, partitionName));
+          } else {
+            String description = String
+                .format("partition=%s, expected=%s, found=%s, %s", partitionName,
+                    idealStateStatus, currentStateStatus, descriptionSuffix);
+            return new Pair(Status.STARTING, description);
+          }
+        }
       }
+      return new Pair(Status.GOOD, STATUS_DESCRIPTION_NONE);
+    }
 
-      if (_resourcesToMonitor.isEmpty()) {
-        _statusDescription = STATUS_DESCRIPTION_NONE;
-        LOGGER.info("Instance {} has finished starting up", _instanceName);
-      } else {
-        _statusDescription = String.format("waitingFor=%s, numResourcesLeft=%d, numTotalResources=%d, minStartCount=%d",
-            getMatchName(), _resourcesToMonitor.size(), _numTotalResourcesToMonitor, _minResourcesStartCount);
-        LOGGER.info("Instance {} returning GOOD because {}", _statusDescription);
+    private String getResourceListAsString() {
+      if (_resourcesToMonitor.size() <= MAX_RESOURCE_NAMES_TO_LOG) {
+        return _resourcesToMonitor.toString();
       }
-
-      return Status.GOOD;
+      return "[" + _resourcesToMonitor.iterator().next() + ",...]";
 
 Review comment:
   I am not clear about this method - why is this moving the resource iterator? It sounds its mostly a toString that restricts it to less than 5 resources logged...

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org