Repository: camel
Updated Branches:
  refs/heads/master 637e2b6ce -> 5711a5750


Refactor BackOff timer


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/5711a575
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/5711a575
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/5711a575

Branch: refs/heads/master
Commit: 5711a5750a718bf915cbf3a0c38cce76e8873711
Parents: 637e2b6
Author: lburgazzoli <lburgazz...@gmail.com>
Authored: Wed Aug 9 19:16:00 2017 +0200
Committer: lburgazzoli <lburgazz...@gmail.com>
Committed: Wed Aug 9 19:16:00 2017 +0200

----------------------------------------------------------------------
 .../camel/impl/SupervisingRouteController.java  |  14 +-
 .../camel/util/backoff/BackOffContext.java      | 179 ---------------
 .../apache/camel/util/backoff/BackOffTimer.java | 143 ++++--------
 .../camel/util/backoff/BackOffTimerTask.java    | 215 +++++++++++++++++++
 .../apache/camel/util/backoff/BackOffTest.java  |   8 +-
 .../camel/util/backoff/BackOffTimerTest.java    |   8 +-
 .../SupervisingRouteControllerRestartTest.java  |   4 +-
 7 files changed, 277 insertions(+), 294 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/5711a575/camel-core/src/main/java/org/apache/camel/impl/SupervisingRouteController.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/main/java/org/apache/camel/impl/SupervisingRouteController.java
 
b/camel-core/src/main/java/org/apache/camel/impl/SupervisingRouteController.java
index a9535e3..45c3e31 100644
--- 
a/camel-core/src/main/java/org/apache/camel/impl/SupervisingRouteController.java
+++ 
b/camel-core/src/main/java/org/apache/camel/impl/SupervisingRouteController.java
@@ -53,7 +53,6 @@ import org.apache.camel.spi.RoutePolicyFactory;
 import org.apache.camel.support.EventNotifierSupport;
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.backoff.BackOff;
-import org.apache.camel.util.backoff.BackOffContext;
 import org.apache.camel.util.backoff.BackOffTimer;
 import org.apache.camel.util.function.ThrowingConsumer;
 import org.slf4j.Logger;
@@ -190,7 +189,7 @@ public class SupervisingRouteController extends 
DefaultRouteController {
         return Collections.unmodifiableList(filters);
     }
 
-    public Optional<BackOffContext> getBackOffContext(String id) {
+    public Optional<BackOffTimer.Task> getBackOffContext(String id) {
         return routeManager.getBackOffContext(id);
     }
 
@@ -467,8 +466,8 @@ public class SupervisingRouteController extends 
DefaultRouteController {
                         }
                     });
 
-                    task.whenComplete((context, throwable) -> {
-                        if (context == null || context.getStatus() != 
BackOffContext.Status.Active) {
+                    task.whenComplete((backOffTask, throwable) -> {
+                        if (backOffTask == null || backOffTask.getStatus() != 
BackOffTimer.Task.Status.Active) {
                             // This indicates that the task has been cancelled
                             // or that back-off retry is exhausted thus if the
                             // route is not started it is moved out of the
@@ -478,7 +477,7 @@ public class SupervisingRouteController extends 
DefaultRouteController {
                                 final ServiceStatus status = route.getStatus();
                                 final boolean stopped = status.isStopped() || 
status.isStopping();
 
-                                if (context != null && context.getStatus() == 
BackOffContext.Status.Exhausted && stopped) {
+                                if (backOffTask != null && 
backOffTask.getStatus() == BackOffTimer.Task.Status.Exhausted && stopped) {
                                     LOGGER.info("Back-off for route {} is 
exhausted, no more attempts will be made and stop supervising it", 
route.getId());
                                     r.getContext().setRouteController(null);
                                 }
@@ -508,12 +507,11 @@ public class SupervisingRouteController extends 
DefaultRouteController {
             routes.clear();
         }
 
-        public Optional<BackOffContext> getBackOffContext(String id) {
+        public Optional<BackOffTimer.Task> getBackOffContext(String id) {
             return routes.entrySet().stream()
                 .filter(e -> ObjectHelper.equal(e.getKey().getId(), id))
                 .findFirst()
-                .map(Map.Entry::getValue)
-                .map(BackOffTimer.Task::getContext);
+                .map(Map.Entry::getValue);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/camel/blob/5711a575/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffContext.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffContext.java 
b/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffContext.java
deleted file mode 100644
index 2358d13..0000000
--- a/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffContext.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/**
- * 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.camel.util.backoff;
-
-/**
- * The context associated to a back-off operation.
- */
-public final class BackOffContext {
-    public enum Status {
-        Active,
-        Inactive,
-        Exhausted
-    }
-
-    private final BackOff backOff;
-
-    private Status status;
-    private long currentAttempts;
-    private long currentDelay;
-    private long currentElapsedTime;
-    private long lastAttemptTime;
-    private long nextAttemptTime;
-
-    public BackOffContext(BackOff backOff) {
-        this.backOff = backOff;
-        this.status = Status.Active;
-
-        this.currentAttempts = 0;
-        this.currentDelay = backOff.getDelay().toMillis();
-        this.currentElapsedTime = 0;
-        this.lastAttemptTime = BackOff.NEVER;
-        this.nextAttemptTime = BackOff.NEVER;
-    }
-
-    // *************************************
-    // Properties
-    // *************************************
-
-    /**
-     * The back-off associated with this context.
-     */
-    public BackOff getBackOff() {
-        return backOff;
-    }
-
-    /**
-     * Gets the context status.
-     */
-    public Status getStatus() {
-        return status;
-    }
-
-    /**
-     * The number of attempts so far.
-     */
-    public long getCurrentAttempts() {
-        return currentAttempts;
-    }
-
-    /**
-     * The current computed delay.
-     */
-    public long getCurrentDelay() {
-        return currentDelay;
-    }
-
-    /**
-     * The current elapsed time.
-     */
-    public long getCurrentElapsedTime() {
-        return currentElapsedTime;
-    }
-
-    /**
-     * The time the last attempt has been performed.
-     */
-    public long getLastAttemptTime() {
-        return lastAttemptTime;
-    }
-
-    /**
-     * Used by BackOffTimer
-     */
-    void setLastAttemptTime(long lastAttemptTime) {
-        this.lastAttemptTime = lastAttemptTime;
-    }
-
-    /**
-     * An indication about the time the next attempt will be made.
-     */
-    public long getNextAttemptTime() {
-        return nextAttemptTime;
-    }
-
-    /**
-     * Used by BackOffTimer
-     */
-    void setNextAttemptTime(long nextAttemptTime) {
-        this.nextAttemptTime = nextAttemptTime;
-    }
-
-    // *************************************
-    // Impl
-    // *************************************
-
-    /**
-     * Return the number of milliseconds to wait before retrying the operation
-     * or ${@link BackOff#NEVER} to indicate that no further attempt should be
-     * made.
-     */
-    long next() {
-        // A call to next when currentDelay is set to NEVER has no effects
-        // as this means that either the timer is exhausted or it has explicit
-        // stopped
-        if (status == Status.Active) {
-
-            currentAttempts++;
-
-            if (currentAttempts > backOff.getMaxAttempts()) {
-                currentDelay = BackOff.NEVER;
-                status = Status.Exhausted;
-            } else if (currentElapsedTime > 
backOff.getMaxElapsedTime().toMillis()) {
-                currentDelay = BackOff.NEVER;
-                status = Status.Exhausted;
-            } else {
-                if (currentDelay <= backOff.getMaxDelay().toMillis()) {
-                    currentDelay = (long) (currentDelay * 
backOff.getMultiplier());
-                }
-
-                currentElapsedTime += currentDelay;
-            }
-        }
-
-        return currentDelay;
-    }
-
-    /**
-     * Reset the context.
-     */
-    public BackOffContext reset() {
-        this.currentAttempts = 0;
-        this.currentDelay = 0;
-        this.currentElapsedTime = 0;
-        this.lastAttemptTime = BackOff.NEVER;
-        this.nextAttemptTime = BackOff.NEVER;
-        this.status = Status.Active;
-
-        return this;
-    }
-
-    /**
-     * Mark the context as exhausted to indicate that no further attempt should
-     * be made.
-     */
-    public BackOffContext stop() {
-        this.currentAttempts = 0;
-        this.currentDelay = BackOff.NEVER;
-        this.currentElapsedTime = 0;
-        this.lastAttemptTime = BackOff.NEVER;
-        this.nextAttemptTime = BackOff.NEVER;
-        this.status = Status.Inactive;
-
-        return this;
-    }
-}

http://git-wip-us.apache.org/repos/asf/camel/blob/5711a575/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffTimer.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffTimer.java 
b/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffTimer.java
index b93bbb1..ae6fc0d 100644
--- a/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffTimer.java
+++ b/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffTimer.java
@@ -16,25 +16,17 @@
  */
 package org.apache.camel.util.backoff;
 
-import java.util.ArrayList;
-import java.util.List;
 import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
 
 import org.apache.camel.util.function.ThrowingFunction;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * A simple timer utility that use a linked {@link BackOff} to determine when
  * a task should be executed.
  */
 public class BackOffTimer {
-    private static final Logger LOGGER = 
LoggerFactory.getLogger(BackOffTimer.class);
-
     private final ScheduledExecutorService scheduler;
 
     public BackOffTimer(ScheduledExecutorService scheduler) {
@@ -45,14 +37,14 @@ public class BackOffTimer {
      * Schedule the given function/task to be executed some time in the future
      * according to the given backOff.
      */
-    public Task schedule(BackOff backOff, ThrowingFunction<BackOffContext, 
Boolean, Exception> function) {
-        final TaskImpl task = new TaskImpl(backOff, function);
+    public Task schedule(BackOff backOff, ThrowingFunction<Task, Boolean, 
Exception> function) {
+        final BackOffTimerTask task = new BackOffTimerTask(backOff, scheduler, 
function);
 
-        long delay = task.getContext().next();
+        long delay = task.next();
         if (delay != BackOff.NEVER) {
             scheduler.schedule(task, delay, TimeUnit.MILLISECONDS);
         } else {
-            task.complete();
+            task.cancel();
         }
 
         return task;
@@ -63,106 +55,63 @@ public class BackOffTimer {
     // ****************************************
 
     public interface Task {
+        enum Status {
+            Active,
+            Inactive,
+            Exhausted
+        }
+
         /**
-         * Gets the {@link BackOffContext} associated with this task.
+         * The back-off associated with this task.
          */
-        BackOffContext getContext();
+        BackOff getBackOff();
 
         /**
-         * Cancel the task.
+         * Gets the task status.
          */
-        void cancel();
+        Status getStatus();
 
         /**
-         * Action to execute when the context is completed (cancelled or 
exhausted)
-         *
-         * @param whenCompleted the consumer.
+         * The number of attempts so far.
          */
-        void whenComplete(BiConsumer<BackOffContext, Throwable> whenCompleted);
-    }
+        long getCurrentAttempts();
 
-    // ****************************************
-    // TimerTask
-    // ****************************************
-
-    private final class TaskImpl implements Task, Runnable {
-        private final BackOffContext context;
-        private final ThrowingFunction<BackOffContext, Boolean, Exception> 
function;
-        private final AtomicReference<ScheduledFuture<?>> futureRef;
-        private final List<BiConsumer<BackOffContext, Throwable>> consumers;
-
-        TaskImpl(BackOff backOff, ThrowingFunction<BackOffContext, Boolean, 
Exception> function) {
-            this.context = new BackOffContext(backOff);
-            this.function = function;
-            this.consumers = new ArrayList<>();
-            this.futureRef = new AtomicReference<>();
-        }
+        /**
+         * The current computed delay.
+         */
+        long getCurrentDelay();
 
-        @Override
-        public void run() {
-            if (context.getStatus() == BackOffContext.Status.Active) {
-                try {
-                    final long currentTime = System.currentTimeMillis();
-
-                    context.setLastAttemptTime(currentTime);
-
-                    if (function.apply(context)) {
-                        long delay = context.next();
-                        if (context.getStatus() != 
BackOffContext.Status.Active) {
-                            // if the call to next makes the context not more
-                            // active, signal task completion.
-                            complete();
-                        } else {
-                            context.setNextAttemptTime(currentTime + delay);
-
-                            // Cache the scheduled future so it can be 
cancelled
-                            // later by Task.cancel()
-                            futureRef.lazySet(scheduler.schedule(this, delay, 
TimeUnit.MILLISECONDS));
-                        }
-                    } else {
-                        // if the function return false no more attempts should
-                        // be made so stop the context.
-                        context.stop();
-
-                        // and signal the task as completed.
-                        complete();
-                    }
-                } catch (Exception e) {
-                    context.stop();
-                    consumers.forEach(c -> c.accept(context, e));
-                }
-            }
-        }
+        /**
+         * The current elapsed time.
+         */
+        long getCurrentElapsedTime();
 
-        @Override
-        public BackOffContext getContext() {
-            return context;
-        }
+        /**
+         * The time the last attempt has been performed.
+         */
+        long getLastAttemptTime();
 
-        @Override
-        public void cancel() {
-            context.stop();
 
-            ScheduledFuture<?> future = futureRef.get();
-            if (future != null) {
-                future.cancel(true);
-            }
+        /**
+         * An indication about the time the next attempt will be made.
+         */
+        long getNextAttemptTime();
 
-            // signal task completion on cancel.
-            complete();
-        }
+        /**
+         * Reset the task.
+         */
+        void reset();
 
-        @Override
-        public void whenComplete(BiConsumer<BackOffContext, Throwable> 
whenCompleted) {
-            synchronized (this.consumers) {
-                consumers.add(whenCompleted);
-            }
-        }
+        /**
+         * Cancel the task.
+         */
+        void cancel();
 
-        void complete() {
-            synchronized (this.consumers) {
-                consumers.forEach(c -> c.accept(context, null));
-            }
-        }
+        /**
+         * Action to execute when the context is completed (cancelled or 
exhausted)
+         *
+         * @param whenCompleted the consumer.
+         */
+        void whenComplete(BiConsumer<Task, Throwable> whenCompleted);
     }
 }

http://git-wip-us.apache.org/repos/asf/camel/blob/5711a575/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffTimerTask.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffTimerTask.java 
b/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffTimerTask.java
new file mode 100644
index 0000000..7c3a505
--- /dev/null
+++ 
b/camel-core/src/main/java/org/apache/camel/util/backoff/BackOffTimerTask.java
@@ -0,0 +1,215 @@
+/**
+ * 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.camel.util.backoff;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BiConsumer;
+
+import org.apache.camel.util.function.ThrowingFunction;
+
+final class BackOffTimerTask implements BackOffTimer.Task, Runnable {
+    private final BackOff backOff;
+    private final ScheduledExecutorService scheduler;
+    private final ThrowingFunction<BackOffTimer.Task, Boolean, Exception> 
function;
+    private final AtomicReference<ScheduledFuture<?>> futureRef;
+    private final List<BiConsumer<BackOffTimer.Task, Throwable>> consumers;
+
+    private Status status;
+    private long currentAttempts;
+    private long currentDelay;
+    private long currentElapsedTime;
+    private long lastAttemptTime;
+    private long nextAttemptTime;
+
+    BackOffTimerTask(BackOff backOff, ScheduledExecutorService scheduler, 
ThrowingFunction<BackOffTimer.Task, Boolean, Exception> function) {
+        this.backOff = backOff;
+        this.scheduler = scheduler;
+        this.status = Status.Active;
+
+        this.currentAttempts = 0;
+        this.currentDelay = backOff.getDelay().toMillis();
+        this.currentElapsedTime = 0;
+        this.lastAttemptTime = BackOff.NEVER;
+        this.nextAttemptTime = BackOff.NEVER;
+
+        this.function = function;
+        this.consumers = new ArrayList<>();
+        this.futureRef = new AtomicReference<>();
+    }
+
+    // *****************************
+    // Properties
+    // *****************************
+
+    @Override
+    public BackOff getBackOff() {
+        return backOff;
+    }
+
+    @Override
+    public Status getStatus() {
+        return status;
+    }
+
+    @Override
+    public long getCurrentAttempts() {
+        return currentAttempts;
+    }
+
+    @Override
+    public long getCurrentDelay() {
+        return currentDelay;
+    }
+
+    @Override
+    public long getCurrentElapsedTime() {
+        return currentElapsedTime;
+    }
+
+    @Override
+    public long getLastAttemptTime() {
+        return lastAttemptTime;
+    }
+
+    @Override
+    public long getNextAttemptTime() {
+        return nextAttemptTime;
+    }
+
+    @Override
+    public void reset() {
+        this.currentAttempts = 0;
+        this.currentDelay = 0;
+        this.currentElapsedTime = 0;
+        this.lastAttemptTime = BackOff.NEVER;
+        this.nextAttemptTime = BackOff.NEVER;
+        this.status = Status.Active;
+    }
+
+    @Override
+    public void cancel() {
+        stop();
+
+        ScheduledFuture<?> future = futureRef.get();
+        if (future != null) {
+            future.cancel(true);
+        }
+
+        // signal task completion on cancel.
+        complete(null);
+    }
+
+    @Override
+    public void whenComplete(BiConsumer<BackOffTimer.Task, Throwable> 
whenCompleted) {
+        synchronized (this.consumers) {
+            consumers.add(whenCompleted);
+        }
+    }
+
+    // *****************************
+    // Task execution
+    // *****************************
+
+    @Override
+    public void run() {
+        if (status == Status.Active) {
+            try {
+                lastAttemptTime = System.currentTimeMillis();
+
+                if (function.apply(this)) {
+                    long delay = next();
+                    if (status != Status.Active) {
+                        // if the call to next makes the context not more
+                        // active, signal task completion.
+                        complete(null);
+                    } else {
+                        nextAttemptTime = lastAttemptTime + delay;
+
+                        // Cache the scheduled future so it can be cancelled
+                        // later by Task.cancel()
+                        futureRef.lazySet(scheduler.schedule(this, delay, 
TimeUnit.MILLISECONDS));
+                    }
+                } else {
+                    stop();
+
+                    // if the function return false no more attempts should
+                    // be made so stop the context.
+                    complete(null);
+                }
+            } catch (Exception e) {
+                stop();
+
+                complete(e);
+            }
+        }
+    }
+
+    void stop() {
+        this.currentAttempts = 0;
+        this.currentDelay = BackOff.NEVER;
+        this.currentElapsedTime = 0;
+        this.lastAttemptTime = BackOff.NEVER;
+        this.nextAttemptTime = BackOff.NEVER;
+        this.status = Status.Inactive;
+    }
+
+    void complete(Throwable throwable) {
+        synchronized (this.consumers) {
+            consumers.forEach(c -> c.accept(this, throwable));
+        }
+    }
+
+    // *****************************
+    // Impl
+    // *****************************
+
+    /**
+     * Return the number of milliseconds to wait before retrying the operation
+     * or ${@link BackOff#NEVER} to indicate that no further attempt should be
+     * made.
+     */
+    long next() {
+        // A call to next when currentDelay is set to NEVER has no effects
+        // as this means that either the timer is exhausted or it has explicit
+        // stopped
+        if (status == Status.Active) {
+
+            currentAttempts++;
+
+            if (currentAttempts > backOff.getMaxAttempts()) {
+                currentDelay = BackOff.NEVER;
+                status = Status.Exhausted;
+            } else if (currentElapsedTime > 
backOff.getMaxElapsedTime().toMillis()) {
+                currentDelay = BackOff.NEVER;
+                status = Status.Exhausted;
+            } else {
+                if (currentDelay <= backOff.getMaxDelay().toMillis()) {
+                    currentDelay = (long) (currentDelay * 
backOff.getMultiplier());
+                }
+
+                currentElapsedTime += currentDelay;
+            }
+        }
+
+        return currentDelay;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/5711a575/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTest.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTest.java 
b/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTest.java
index aa18956..0d5f525 100644
--- a/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTest.java
+++ b/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTest.java
@@ -26,7 +26,7 @@ public class BackOffTest {
     @Test
     public void testSimpleBackOff() {
         final BackOff backOff = BackOff.builder().build();
-        final BackOffContext context = new BackOffContext(backOff);
+        final BackOffTimerTask context = new BackOffTimerTask(backOff, null, t 
-> true);
 
         long delay;
 
@@ -42,7 +42,7 @@ public class BackOffTest {
     @Test
     public void testBackOffWithMultiplier() {
         final BackOff backOff = BackOff.builder().multiplier(1.5).build();
-        final BackOffContext context = new BackOffContext(backOff);
+        final BackOffTimerTask context = new BackOffTimerTask(backOff, null, t 
-> true);
 
         long delay = BackOff.DEFAULT_DELAY.toMillis();
         long oldDelay;
@@ -63,7 +63,7 @@ public class BackOffTest {
     @Test
     public void testBackOffWithMaxAttempts() {
         final BackOff backOff = BackOff.builder().maxAttempts(5L).build();
-        final BackOffContext context = new BackOffContext(backOff);
+        final BackOffTimerTask context = new BackOffTimerTask(backOff, null, t 
-> true);
 
         long delay;
 
@@ -83,7 +83,7 @@ public class BackOffTest {
     @Test
     public void testBackOffWithMaxTime() {
         final BackOff backOff = BackOff.builder().maxElapsedTime(9, 
TimeUnit.SECONDS).build();
-        final BackOffContext context = new BackOffContext(backOff);
+        final BackOffTimerTask context = new BackOffTimerTask(backOff, null, t 
-> true);
 
         long delay;
 

http://git-wip-us.apache.org/repos/asf/camel/blob/5711a575/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTimerTest.java
----------------------------------------------------------------------
diff --git 
a/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTimerTest.java 
b/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTimerTest.java
index 7d9f0e7..30c9b31 100644
--- 
a/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTimerTest.java
+++ 
b/camel-core/src/test/java/org/apache/camel/util/backoff/BackOffTimerTest.java
@@ -81,7 +81,7 @@ public class BackOffTimerTest {
         task.whenComplete(
             (context, throwable) -> {
                 Assert.assertEquals(5, counter.get());
-                Assert.assertEquals(BackOffContext.Status.Exhausted, 
context.getStatus());
+                Assert.assertEquals(BackOffTimer.Task.Status.Exhausted, 
context.getStatus());
                 latch.countDown();
             }
         );
@@ -113,7 +113,7 @@ public class BackOffTimerTest {
         task.whenComplete(
             (context, throwable) -> {
                 Assert.assertTrue(counter.get() <= 5);
-                Assert.assertEquals(BackOffContext.Status.Exhausted, 
context.getStatus());
+                Assert.assertEquals(BackOffTimer.Task.Status.Exhausted, 
context.getStatus());
                 latch.countDown();
             }
         );
@@ -133,7 +133,7 @@ public class BackOffTimerTest {
         BackOffTimer.Task task = timer.schedule(
             backOff,
             context -> {
-                Assert.assertEquals(BackOffContext.Status.Active, 
context.getStatus());
+                Assert.assertEquals(BackOffTimer.Task.Status.Active, 
context.getStatus());
 
                 latch.countDown();
 
@@ -143,7 +143,7 @@ public class BackOffTimerTest {
 
         task.whenComplete(
             (context, throwable) -> {
-                Assert.assertEquals(BackOffContext.Status.Inactive, 
context.getStatus());
+                Assert.assertEquals(BackOffTimer.Task.Status.Inactive, 
context.getStatus());
                 done.set(true);
             }
         );

http://git-wip-us.apache.org/repos/asf/camel/blob/5711a575/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerRestartTest.java
----------------------------------------------------------------------
diff --git 
a/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerRestartTest.java
 
b/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerRestartTest.java
index cbe7b2b..77547ab 100644
--- 
a/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerRestartTest.java
+++ 
b/components/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerRestartTest.java
@@ -24,7 +24,7 @@ import org.apache.camel.ServiceStatus;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.impl.SupervisingRouteController;
 import org.apache.camel.test.AvailablePortFinder;
-import org.apache.camel.util.backoff.BackOffContext;
+import org.apache.camel.util.backoff.BackOffTimer;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -96,7 +96,7 @@ public class SupervisingRouteControllerRestartTest {
         Thread.sleep(2000);
 
         Assert.assertTrue(controller.getBackOffContext("jetty").isPresent());
-        Assert.assertEquals(BackOffContext.Status.Active, 
controller.getBackOffContext("jetty").get().getStatus());
+        Assert.assertEquals(BackOffTimer.Task.Status.Active, 
controller.getBackOffContext("jetty").get().getStatus());
         
Assert.assertTrue(controller.getBackOffContext("jetty").get().getCurrentAttempts()
 > 0);
 
         try {

Reply via email to