You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "slfan1989 (via GitHub)" <gi...@apache.org> on 2023/05/21 10:50:09 UTC

[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5676: YARN-6648. BackPort [GPG] Add SubClusterCleaner in Global Policy Generator.

slfan1989 commented on code in PR #5676:
URL: https://github.com/apache/hadoop/pull/5676#discussion_r1199742041


##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java:
##########
@@ -4326,6 +4326,24 @@ public static boolean isAclEnabled(Configuration conf) {
   public static final boolean DEFAULT_ROUTER_WEBAPP_PARTIAL_RESULTS_ENABLED =
       false;
 
+  private static final String FEDERATION_GPG_PREFIX =
+          FEDERATION_PREFIX + "gpg.";
+
+  // The number of threads to use for the GPG scheduled executor service
+  public static final String GPG_SCHEDULED_EXECUTOR_THREADS =
+          FEDERATION_GPG_PREFIX + "scheduled.executor.threads";
+  public static final int DEFAULT_GPG_SCHEDULED_EXECUTOR_THREADS = 10;
+
+  // The interval at which the subcluster cleaner runs, -1 means disabled
+  public static final String GPG_SUBCLUSTER_CLEANER_INTERVAL_MS =
+          FEDERATION_GPG_PREFIX + "subcluster.cleaner.interval-ms";
+  public static final long DEFAULT_GPG_SUBCLUSTER_CLEANER_INTERVAL_MS = -1;
+
+  // The expiration time for a subcluster heartbeat, default is 30 minutes
+  public static final String GPG_SUBCLUSTER_EXPIRATION_MS =
+          FEDERATION_GPG_PREFIX + "subcluster.heartbeat.expiration-ms";
+  public static final long DEFAULT_GPG_SUBCLUSTER_EXPIRATION_MS = 1800000;

Review Comment:
   Thank you for helping to review the code, I will improve the code.



##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-globalpolicygenerator/src/main/java/org/apache/hadoop/yarn/server/globalpolicygenerator/subclustercleaner/SubClusterCleaner.java:
##########
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.globalpolicygenerator.subclustercleaner;
+
+import java.util.Date;
+import java.util.Map;
+
+import org.apache.commons.lang.time.DurationFormatUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.globalpolicygenerator.GPGContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The sub-cluster cleaner is one of the GPG's services that periodically checks
+ * the membership table in FederationStateStore and mark sub-clusters that have
+ * not sent a heartbeat in certain amount of time as LOST.
+ */
+public class SubClusterCleaner implements Runnable {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SubClusterCleaner.class);
+
+  private GPGContext gpgContext;
+  private long heartbeatExpirationMillis;
+
+  /**
+   * The sub-cluster cleaner runnable is invoked by the sub cluster cleaner
+   * service to check the membership table and remove sub clusters that have not
+   * sent a heart beat in some amount of time.
+   *
+   * @param conf configuration.
+   * @param gpgContext GPGContext.
+   */
+  public SubClusterCleaner(Configuration conf, GPGContext gpgContext) {
+    this.heartbeatExpirationMillis =
+        conf.getLong(YarnConfiguration.GPG_SUBCLUSTER_EXPIRATION_MS,

Review Comment:
   I will improve the code.



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

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

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


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