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 2021/06/03 05:56:08 UTC

[GitHub] [ozone] bharatviswa504 opened a new pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN.

bharatviswa504 opened a new pull request #2301:
URL: https://github.com/apache/ozone/pull/2301


   ## What changes were proposed in this pull request?
   
   Handle SIGTERM in OM/DN
   Fixed an issue in shutdown hook of OM, where it is failing if a directory already exists. Added a check.
   And also increased time out wait in ozone scripts, as DN stop is taking more than 5 seconds during local test. And I see it is doing kill -9. To avoid this increased to 30sec, which is default wait of shutdown hook.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5291
   
   ## How was this patch tested?
   
   Tested locally.
   


-- 
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



---------------------------------------------------------------------
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 change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r645024087



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,381 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook
+ * in a deterministic order, higher priority first.
+ * <p/>
+ * The JVM runs ShutdownHooks in a non-deterministic order or in parallel.
+ * This class registers a single JVM shutdownHook and run all the
+ * shutdownHooks registered to it (to this class) in order based on their
+ * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link org.apache.hadoop.ozone.OzoneConfigKeys#SERVICE_SHUTDOWN_TIMEOUT} in
+ * {@code ozone-site.xml}, with a default value of
+ * {@link org.apache.hadoop.ozone.OzoneConfigKeys
+ * #SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
+
+  private static final ShutdownHookManager MGR = new ShutdownHookManager();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(org.apache.hadoop.util.ShutdownHookManager.class);

Review comment:
       Logger references Hadoop's class instead of this one.
   
   ```suggestion
         LoggerFactory.getLogger(ShutdownHookManager.class);
   ```




-- 
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



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


[GitHub] [ozone] bharatviswa504 commented on pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#issuecomment-865612374


   @xiaoyuyao / @adoroszlai Updated patch, PTAL.


-- 
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



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r665598690



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,389 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.annotation.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook
+ * in a deterministic order, higher priority first.
+ * <p/>
+ * The JVM runs ShutdownHooks in a non-deterministic order or in parallel.
+ * This class registers a single JVM shutdownHook and run all the
+ * shutdownHooks registered to it (to this class) in order based on their
+ * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link org.apache.hadoop.ozone.conf.OzoneServiceConfig
+ * #SERVICE_SHUTDOWN_TIMEOUT}
+ * {@code ozone-site.xml}, with a default value of
+ * {@link org.apache.hadoop.ozone.conf.OzoneServiceConfig
+ * #SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
+ *
+ * This code is taken from hadoop project.
+ * 1. This is to avoid dependency on hadoop.
+ * 2. To use a ozone specific config and defaults.
+ * 3. Now any fix happened to this class in hadoop for this class, we can
+ * backport this to this with out waiting for a new hadoop release.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
+
+  private static final ShutdownHookManager MGR = new ShutdownHookManager();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ShutdownHookManager.class);
+
+  /** Minimum shutdown timeout: {@value} second(s). */
+  public static final long TIMEOUT_MINIMUM = 1;
+
+  /** The default time unit used: seconds. */
+  public static final TimeUnit TIME_UNIT_DEFAULT = TimeUnit.SECONDS;
+
+  private static final ExecutorService EXECUTOR =
+      HadoopExecutors.newSingleThreadExecutor(new ThreadFactoryBuilder()

Review comment:
       Do we want to remove the dependency on HadoopExecutors as well? This can be a separate JIRA.




-- 
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] bharatviswa504 commented on pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#issuecomment-865612374


   @xiaoyuyao / @adoroszlai Updated patch, PTAL.


-- 
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



---------------------------------------------------------------------
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 change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r644980793



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,381 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

Review comment:
       Please use Ozone-specific annotations:
   
   ```suggestion
   import org.apache.hadoop.hdds.annotation.InterfaceAudience;
   import org.apache.hadoop.hdds.annotation.InterfaceStability;
   ```

##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,381 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook
+ * in a deterministic order, higher priority first.
+ * <p/>
+ * The JVM runs ShutdownHooks in a non-deterministic order or in parallel.
+ * This class registers a single JVM shutdownHook and run all the
+ * shutdownHooks registered to it (to this class) in order based on their
+ * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link org.apache.hadoop.ozone.OzoneConfigKeys#SERVICE_SHUTDOWN_TIMEOUT} in
+ * {@code ozone-site.xml}, with a default value of
+ * {@link org.apache.hadoop.ozone.OzoneConfigKeys
+ * #SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
+
+  private static final ShutdownHookManager MGR = new ShutdownHookManager();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(org.apache.hadoop.util.ShutdownHookManager.class);

Review comment:
       ```suggestion
         LoggerFactory.getLogger(ShutdownHookManager.class);
   ```

##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
##########
@@ -449,6 +449,15 @@
   public static final long OZONE_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT =
       TimeUnit.DAYS.toMillis(10); // 10 days
 
+
+  public static final String SERVICE_SHUTDOWN_TIMEOUT =
+      "ozone.service.shutdown.timeout";
+  /** Default shutdown hook timeout: {@value} seconds. */
+  public static final long SERVICE_SHUTDOWN_TIMEOUT_DEFAULT = 60;
+
+  public static final TimeUnit SERVICE_SHUTDOWN_TIME_UNIT_DEFAULT =
+      TimeUnit.SECONDS;

Review comment:
       I think this is a great candidate to define in typesafe way (using `@Config` annotation).




-- 
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



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


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r649780508



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
##########
@@ -449,6 +449,15 @@
   public static final long OZONE_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT =
       TimeUnit.DAYS.toMillis(10); // 10 days
 
+
+  public static final String SERVICE_SHUTDOWN_TIMEOUT =
+      "ozone.service.shutdown.timeout";
+  /** Default shutdown hook timeout: {@value} seconds. */
+  public static final long SERVICE_SHUTDOWN_TIMEOUT_DEFAULT = 60;
+
+  public static final TimeUnit SERVICE_SHUTDOWN_TIME_UNIT_DEFAULT =
+      TimeUnit.SECONDS;

Review comment:
       Done

##########
File path: hadoop-hdds/common/src/main/resources/ozone-default.xml
##########
@@ -2863,4 +2863,23 @@
     </description>
   </property>
 
+
+  <property>
+    <name>ozone.service.shutdown.timeout</name>
+    <tag>OZONE, SCM, OM, DATANODE</tag>
+    <value>60s</value>
+    <description>
+      Timeout to wait for each shutdown operation to complete.
+      If a hook takes longer than this time to complete, it will be interrupted,
+      so the service will shutdown. This allows the service shutdown
+      to recover from a blocked operation.
+      Some shutdown hooks may need more time than this, for example when DN

Review comment:
       Removed this.

##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,381 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook

Review comment:
       Added a little details why it is done.




-- 
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



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r645022340



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,381 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook

Review comment:
       This seems to be the same of the one from hadoop ShutdownHookManager? If it is to avoid dependency, can you add some context forking the code here so that in the future we can port hadoop ShutdownHookManager bug fix easier?




-- 
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



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


[GitHub] [ozone] bharatviswa504 commented on pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#issuecomment-878179070


   Thank You @xiaoyuyao for the review.
   Addressed review 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] bharatviswa504 commented on pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#issuecomment-859375539


   Thank You @xiaoyuyao and @adoroszlai for the review.
   Fixed review 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.

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 change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r644980793



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,381 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

Review comment:
       Please use Ozone-specific annotations:
   
   ```suggestion
   import org.apache.hadoop.hdds.annotation.InterfaceAudience;
   import org.apache.hadoop.hdds.annotation.InterfaceStability;
   ```

##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,381 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook
+ * in a deterministic order, higher priority first.
+ * <p/>
+ * The JVM runs ShutdownHooks in a non-deterministic order or in parallel.
+ * This class registers a single JVM shutdownHook and run all the
+ * shutdownHooks registered to it (to this class) in order based on their
+ * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link org.apache.hadoop.ozone.OzoneConfigKeys#SERVICE_SHUTDOWN_TIMEOUT} in
+ * {@code ozone-site.xml}, with a default value of
+ * {@link org.apache.hadoop.ozone.OzoneConfigKeys
+ * #SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
+
+  private static final ShutdownHookManager MGR = new ShutdownHookManager();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(org.apache.hadoop.util.ShutdownHookManager.class);

Review comment:
       ```suggestion
         LoggerFactory.getLogger(ShutdownHookManager.class);
   ```

##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
##########
@@ -449,6 +449,15 @@
   public static final long OZONE_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT =
       TimeUnit.DAYS.toMillis(10); // 10 days
 
+
+  public static final String SERVICE_SHUTDOWN_TIMEOUT =
+      "ozone.service.shutdown.timeout";
+  /** Default shutdown hook timeout: {@value} seconds. */
+  public static final long SERVICE_SHUTDOWN_TIMEOUT_DEFAULT = 60;
+
+  public static final TimeUnit SERVICE_SHUTDOWN_TIME_UNIT_DEFAULT =
+      TimeUnit.SECONDS;

Review comment:
       I think this is a great candidate to define in typesafe way (using `@Config` annotation).

##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,381 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook
+ * in a deterministic order, higher priority first.
+ * <p/>
+ * The JVM runs ShutdownHooks in a non-deterministic order or in parallel.
+ * This class registers a single JVM shutdownHook and run all the
+ * shutdownHooks registered to it (to this class) in order based on their
+ * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link org.apache.hadoop.ozone.OzoneConfigKeys#SERVICE_SHUTDOWN_TIMEOUT} in
+ * {@code ozone-site.xml}, with a default value of
+ * {@link org.apache.hadoop.ozone.OzoneConfigKeys
+ * #SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
+
+  private static final ShutdownHookManager MGR = new ShutdownHookManager();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(org.apache.hadoop.util.ShutdownHookManager.class);

Review comment:
       Logger references Hadoop's class instead of this one.
   
   ```suggestion
         LoggerFactory.getLogger(ShutdownHookManager.class);
   ```




-- 
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



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


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r667828508



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,389 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.annotation.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook
+ * in a deterministic order, higher priority first.
+ * <p/>
+ * The JVM runs ShutdownHooks in a non-deterministic order or in parallel.
+ * This class registers a single JVM shutdownHook and run all the
+ * shutdownHooks registered to it (to this class) in order based on their
+ * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link org.apache.hadoop.ozone.conf.OzoneServiceConfig
+ * #SERVICE_SHUTDOWN_TIMEOUT}
+ * {@code ozone-site.xml}, with a default value of
+ * {@link org.apache.hadoop.ozone.conf.OzoneServiceConfig
+ * #SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
+ *
+ * This code is taken from hadoop project.
+ * 1. This is to avoid dependency on hadoop.
+ * 2. To use a ozone specific config and defaults.
+ * 3. Now any fix happened to this class in hadoop for this class, we can
+ * backport this to this with out waiting for a new hadoop release.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
+
+  private static final ShutdownHookManager MGR = new ShutdownHookManager();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ShutdownHookManager.class);
+
+  /** Minimum shutdown timeout: {@value} second(s). */
+  public static final long TIMEOUT_MINIMUM = 1;

Review comment:
       Done

##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,389 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.annotation.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook
+ * in a deterministic order, higher priority first.
+ * <p/>
+ * The JVM runs ShutdownHooks in a non-deterministic order or in parallel.
+ * This class registers a single JVM shutdownHook and run all the
+ * shutdownHooks registered to it (to this class) in order based on their
+ * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link org.apache.hadoop.ozone.conf.OzoneServiceConfig
+ * #SERVICE_SHUTDOWN_TIMEOUT}
+ * {@code ozone-site.xml}, with a default value of
+ * {@link org.apache.hadoop.ozone.conf.OzoneServiceConfig
+ * #SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
+ *
+ * This code is taken from hadoop project.
+ * 1. This is to avoid dependency on hadoop.
+ * 2. To use a ozone specific config and defaults.
+ * 3. Now any fix happened to this class in hadoop for this class, we can
+ * backport this to this with out waiting for a new hadoop release.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
+
+  private static final ShutdownHookManager MGR = new ShutdownHookManager();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ShutdownHookManager.class);
+
+  /** Minimum shutdown timeout: {@value} second(s). */
+  public static final long TIMEOUT_MINIMUM = 1;
+
+  /** The default time unit used: seconds. */
+  public static final TimeUnit TIME_UNIT_DEFAULT = TimeUnit.SECONDS;
+
+  private static final ExecutorService EXECUTOR =
+      HadoopExecutors.newSingleThreadExecutor(new ThreadFactoryBuilder()

Review comment:
       Done




-- 
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] xiaoyuyao commented on a change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r665593092



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,389 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.hdds.annotation.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook
+ * in a deterministic order, higher priority first.
+ * <p/>
+ * The JVM runs ShutdownHooks in a non-deterministic order or in parallel.
+ * This class registers a single JVM shutdownHook and run all the
+ * shutdownHooks registered to it (to this class) in order based on their
+ * priority.
+ *
+ * Unless a hook was registered with a shutdown explicitly set through
+ * {@link #addShutdownHook(Runnable, int, long, TimeUnit)},
+ * the shutdown time allocated to it is set by the configuration option
+ * {@link org.apache.hadoop.ozone.conf.OzoneServiceConfig
+ * #SERVICE_SHUTDOWN_TIMEOUT}
+ * {@code ozone-site.xml}, with a default value of
+ * {@link org.apache.hadoop.ozone.conf.OzoneServiceConfig
+ * #SERVICE_SHUTDOWN_TIMEOUT_DEFAULT}
+ * seconds.
+ *
+ * This code is taken from hadoop project.
+ * 1. This is to avoid dependency on hadoop.
+ * 2. To use a ozone specific config and defaults.
+ * 3. Now any fix happened to this class in hadoop for this class, we can
+ * backport this to this with out waiting for a new hadoop release.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ShutdownHookManager {
+
+  private static final ShutdownHookManager MGR = new ShutdownHookManager();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ShutdownHookManager.class);
+
+  /** Minimum shutdown timeout: {@value} second(s). */
+  public static final long TIMEOUT_MINIMUM = 1;

Review comment:
       NIT: Can we wrap TIMEOUT_MINIMUM and TIME_UNIT_DEFAULT in OzoneServiceConfig? 




-- 
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] xiaoyuyao commented on a change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r645022340



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/ShutdownHookManager.java
##########
@@ -0,0 +1,381 @@
+/**
+ * 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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The <code>ShutdownHookManager</code> enables running shutdownHook

Review comment:
       This seems to be the same of the one from hadoop ShutdownHookManager? If it is to avoid dependency, can you add some context forking the code here so that in the future we can port hadoop ShutdownHookManager bug fix easier?

##########
File path: hadoop-hdds/common/src/main/resources/ozone-default.xml
##########
@@ -2863,4 +2863,23 @@
     </description>
   </property>
 
+
+  <property>
+    <name>ozone.service.shutdown.timeout</name>
+    <tag>OZONE, SCM, OM, DATANODE</tag>
+    <value>60s</value>
+    <description>
+      Timeout to wait for each shutdown operation to complete.
+      If a hook takes longer than this time to complete, it will be interrupted,
+      so the service will shutdown. This allows the service shutdown
+      to recover from a blocked operation.
+      Some shutdown hooks may need more time than this, for example when DN

Review comment:
       " for example when DN" seems to be incomplete.




-- 
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



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


[GitHub] [ozone] xiaoyuyao commented on pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#issuecomment-881158160


   LGTM, +1. 


-- 
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] xiaoyuyao merged pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
xiaoyuyao merged pull request #2301:
URL: https://github.com/apache/ozone/pull/2301


   


-- 
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] xiaoyuyao commented on a change in pull request #2301: HDDS-5291. Handle SIGTERM to ensure clean shutdown of OM/DN/SCM

Posted by GitBox <gi...@apache.org>.
xiaoyuyao commented on a change in pull request #2301:
URL: https://github.com/apache/ozone/pull/2301#discussion_r645043953



##########
File path: hadoop-hdds/common/src/main/resources/ozone-default.xml
##########
@@ -2863,4 +2863,23 @@
     </description>
   </property>
 
+
+  <property>
+    <name>ozone.service.shutdown.timeout</name>
+    <tag>OZONE, SCM, OM, DATANODE</tag>
+    <value>60s</value>
+    <description>
+      Timeout to wait for each shutdown operation to complete.
+      If a hook takes longer than this time to complete, it will be interrupted,
+      so the service will shutdown. This allows the service shutdown
+      to recover from a blocked operation.
+      Some shutdown hooks may need more time than this, for example when DN

Review comment:
       " for example when DN" seems to be incomplete.




-- 
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



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