Repository: bookkeeper
Updated Branches:
  refs/heads/master 5d282dcea -> 25c113f62


BOOKKEEPER-1046: Avoid long to Long conversion in OrderedSafeExecutor task 
submit

When submitting tasks to an OrderedSafeExecutor, most of the time a ledger id 
is being passed. Given that the method accepts and Object, the primitive `long` 
is boxed into a `Long` allocated on the heap.

Added specific method overload to directly accept longs as the key in the 
OrderedSafeExecutor.

Author: Matteo Merli <[email protected]>

Reviewers: Enrico Olivelli <[email protected]>

Closes #136 from merlimat/executor-long-conversion


Project: http://git-wip-us.apache.org/repos/asf/bookkeeper/repo
Commit: http://git-wip-us.apache.org/repos/asf/bookkeeper/commit/25c113f6
Tree: http://git-wip-us.apache.org/repos/asf/bookkeeper/tree/25c113f6
Diff: http://git-wip-us.apache.org/repos/asf/bookkeeper/diff/25c113f6

Branch: refs/heads/master
Commit: 25c113f62a40bff6ff91cf82f699fd1d35b102fb
Parents: 5d282dc
Author: Matteo Merli <[email protected]>
Authored: Thu May 4 20:54:13 2017 +0200
Committer: Enrico Olivelli <[email protected]>
Committed: Thu May 4 20:54:13 2017 +0200

----------------------------------------------------------------------
 .../bookkeeper/util/OrderedSafeExecutor.java    | 34 +++++++++++++++++---
 1 file changed, 30 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bookkeeper/blob/25c113f6/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/OrderedSafeExecutor.java
----------------------------------------------------------------------
diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/OrderedSafeExecutor.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/OrderedSafeExecutor.java
index 7b4499f..05bd100 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/OrderedSafeExecutor.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/OrderedSafeExecutor.java
@@ -258,7 +258,15 @@ public class OrderedSafeExecutor {
         }
 
         return threads[MathUtils.signSafeMod(orderingKey.hashCode(), 
threads.length)];
+    }
+
+    ExecutorService chooseThread(long orderingKey) {
+        // skip hashcode generation in this special case
+        if (threads.length == 1) {
+            return threads[0];
+        }
 
+        return threads[MathUtils.signSafeMod(orderingKey, threads.length)];
     }
 
     private SafeRunnable timedRunnable(SafeRunnable r) {
@@ -383,13 +391,31 @@ public class OrderedSafeExecutor {
         return chooseThread(orderingKey).scheduleWithFixedDelay(command, 
initialDelay, delay, unit);
     }
 
-    private long getThreadID(Object orderingKey) {
+    /**
+     * schedules a one time action to execute with an ordering guarantee on 
the key
+     * @param orderingKey
+     * @param r
+     */
+    public void submitOrdered(long orderingKey, SafeRunnable r) {
+        chooseThread(orderingKey).submit(r);
+    }
+
+    /**
+     * schedules a one time action to execute with an ordering guarantee on 
the key
+     * @param orderingKey
+     * @param r
+     */
+    public void submitOrdered(int orderingKey, SafeRunnable r) {
+        chooseThread(orderingKey).submit(r);
+    }
+
+    private long getThreadID(long orderingKey) {
         // skip hashcode generation in this special case
         if (threadIds.length == 1) {
             return threadIds[0];
         }
 
-        return threadIds[MathUtils.signSafeMod(orderingKey.hashCode(), 
threadIds.length)];
+        return threadIds[MathUtils.signSafeMod(orderingKey, threadIds.length)];
     }
 
     public void shutdown() {
@@ -415,14 +441,14 @@ public class OrderedSafeExecutor {
         private final Logger LOG = 
LoggerFactory.getLogger(OrderedSafeGenericCallback.class);
 
         private final OrderedSafeExecutor executor;
-        private final Object orderingKey;
+        private final long orderingKey;
 
         /**
          * @param executor The executor on which to run the callback
          * @param orderingKey Key used to decide which thread the callback
          *                    should run on.
          */
-        public OrderedSafeGenericCallback(OrderedSafeExecutor executor, Object 
orderingKey) {
+        public OrderedSafeGenericCallback(OrderedSafeExecutor executor, long 
orderingKey) {
             this.executor = executor;
             this.orderingKey = orderingKey;
         }

Reply via email to