You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/11/17 09:06:52 UTC

[GitHub] [pulsar] lhotari opened a new pull request #12853: Catch exceptions in scheduled tasks to prevent unintended cancellation

lhotari opened a new pull request #12853:
URL: https://github.com/apache/pulsar/pull/12853


   ### Motivation
   
   - Prevent the intended cancellation of scheduled tasks in Pulsar code base.
   - [ScheduledExecutorService#scheduleAtFixedRate](https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/util/concurrent/ScheduledExecutorService.html#scheduleAtFixedRate(java.lang.Runnable,long,long,java.util.concurrent.TimeUnit)) (and [scheduleWithFixedDelay](https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/util/concurrent/ScheduledExecutorService.html#scheduleWithFixedDelay(java.lang.Runnable,long,long,java.util.concurrent.TimeUnit))) won't schedule the next execution if running the task throws an exception. This is explained [in the javadoc](https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/util/concurrent/ScheduledExecutorService.html#scheduleAtFixedRate(java.lang.Runnable,long,long,java.util.concurrent.TimeUnit)). This can lead to unintended cancellation of the scheduled task in failure scenarios.
   
   ### Additional context
   
   This is a cross-cutting change that is about the correct usage of ScheduleExecutorService API. The problem and solution is explained in this StackOverflow Answer: https://stackoverflow.com/a/24902026.
   
   Bookkeeper client includes a solution called SafeRunnable (in 2 locations: in https://github.com/apache/bookkeeper/blob/master/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/SafeRunnable.java and https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/SafeRunnable.java). The SafeRunnable.safeRun method is used extensively in many parts of Bookkeeper code. Some parts of Pulsar code base uses SafeRunnable.safeRun to catch and log exceptions. 
   
   The concept of "SafeRunnable" is confusing. Instead of copying the same solution that Bookkeeper uses, it is intentional that there is no "safe runnable" and the method is called "Runnables.catchingAndLoggingThrowables" to make it explicit that the Runnable is wrapped with handling that catches and logs throwables. There is no "magic".
   
   ### Modifications
   
   - Add new static method `org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables` to pulsar-common which wraps a `Runnable` with try-catch block to catch and log all Throwables.
   - Use this wrapper in most usages of ScheduledExecutorService scheduleAtFixedRate and scheduleWithFixedDelay methods.
   
   
   


-- 
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: commits-unsubscribe@pulsar.apache.org

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



[GitHub] [pulsar] lhotari commented on a change in pull request #12853: Catch exceptions in scheduled tasks to prevent unintended cancellation

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #12853:
URL: https://github.com/apache/pulsar/pull/12853#discussion_r751084661



##########
File path: pulsar-common/src/main/java/org/apache/pulsar/common/util/Runnables.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.pulsar.common.util;
+
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class Runnables {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Runnables.class);
+
+    /**
+     * Wraps a Runnable so that throwables are caught and logged when a Runnable is run.
+     *
+     * The main usecase for this method is to be used in {@link java.util.concurrent.ScheduledExecutorService#scheduleAtFixedRate(Runnable, long, long, TimeUnit)}
+     * calls to ensure that the scheduled task doesn't get cancelled as a result of an uncaught exception.
+     *
+     * @param runnable The runnable to wrap
+     * @return a wrapped Runnable
+     */
+    public static Runnable catchingAndLoggingThrowables(Runnable runnable) {

Review comment:
       It's intentionally long and descriptive and does exactly what the method name says. I explained in the PR description why I think SafeRunnable / safeRun is a bad concept.




-- 
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: commits-unsubscribe@pulsar.apache.org

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



[GitHub] [pulsar] lhotari merged pull request #12853: Catch exceptions in scheduled tasks to prevent unintended cancellation

Posted by GitBox <gi...@apache.org>.
lhotari merged pull request #12853:
URL: https://github.com/apache/pulsar/pull/12853


   


-- 
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: commits-unsubscribe@pulsar.apache.org

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



[GitHub] [pulsar] lhotari commented on a change in pull request #12853: Catch exceptions in scheduled tasks to prevent unintended cancellation

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #12853:
URL: https://github.com/apache/pulsar/pull/12853#discussion_r751085265



##########
File path: pulsar-common/src/main/java/org/apache/pulsar/common/util/Runnables.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.pulsar.common.util;
+
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class Runnables {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Runnables.class);
+
+    /**
+     * Wraps a Runnable so that throwables are caught and logged when a Runnable is run.
+     *
+     * The main usecase for this method is to be used in {@link java.util.concurrent.ScheduledExecutorService#scheduleAtFixedRate(Runnable, long, long, TimeUnit)}
+     * calls to ensure that the scheduled task doesn't get cancelled as a result of an uncaught exception.
+     *
+     * @param runnable The runnable to wrap
+     * @return a wrapped Runnable
+     */
+    public static Runnable catchingAndLoggingThrowables(Runnable runnable) {
+        return () -> {

Review comment:
       Makes sense, I'll change it to use an ordinary static inner 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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

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



[GitHub] [pulsar] lhotari commented on a change in pull request #12853: Catch exceptions in scheduled tasks to prevent unintended cancellation

Posted by GitBox <gi...@apache.org>.
lhotari commented on a change in pull request #12853:
URL: https://github.com/apache/pulsar/pull/12853#discussion_r751084929



##########
File path: pulsar-common/src/main/java/org/apache/pulsar/common/util/Runnables.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.pulsar.common.util;
+
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class Runnables {

Review comment:
       sure, I'll switch to that solution.




-- 
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: commits-unsubscribe@pulsar.apache.org

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



[GitHub] [pulsar] eolivelli commented on a change in pull request #12853: Catch exceptions in scheduled tasks to prevent unintended cancellation

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #12853:
URL: https://github.com/apache/pulsar/pull/12853#discussion_r751079058



##########
File path: pulsar-common/src/main/java/org/apache/pulsar/common/util/Runnables.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.pulsar.common.util;
+
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class Runnables {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Runnables.class);
+
+    /**
+     * Wraps a Runnable so that throwables are caught and logged when a Runnable is run.
+     *
+     * The main usecase for this method is to be used in {@link java.util.concurrent.ScheduledExecutorService#scheduleAtFixedRate(Runnable, long, long, TimeUnit)}
+     * calls to ensure that the scheduled task doesn't get cancelled as a result of an uncaught exception.
+     *
+     * @param runnable The runnable to wrap
+     * @return a wrapped Runnable
+     */
+    public static Runnable catchingAndLoggingThrowables(Runnable runnable) {

Review comment:
       What about 'safeRun'?
   This name sounds too long and verbose to me

##########
File path: pulsar-common/src/main/java/org/apache/pulsar/common/util/Runnables.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.pulsar.common.util;
+
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class Runnables {

Review comment:
       What about adding a private constructor?

##########
File path: pulsar-common/src/main/java/org/apache/pulsar/common/util/Runnables.java
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.pulsar.common.util;
+
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class Runnables {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Runnables.class);
+
+    /**
+     * Wraps a Runnable so that throwables are caught and logged when a Runnable is run.
+     *
+     * The main usecase for this method is to be used in {@link java.util.concurrent.ScheduledExecutorService#scheduleAtFixedRate(Runnable, long, long, TimeUnit)}
+     * calls to ensure that the scheduled task doesn't get cancelled as a result of an uncaught exception.
+     *
+     * @param runnable The runnable to wrap
+     * @return a wrapped Runnable
+     */
+    public static Runnable catchingAndLoggingThrowables(Runnable runnable) {
+        return () -> {

Review comment:
       Lambda usually generate ugly stack traces.
   What about using a regular static inner 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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

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