You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by js...@apache.org on 2007/04/25 16:06:43 UTC

svn commit: r532360 - in /activemq/camel/trunk/camel-core/src: main/java/org/apache/camel/processor/idempotent/ main/java/org/apache/camel/util/ test/java/org/apache/camel/builder/ test/java/org/apache/camel/processor/

Author: jstrachan
Date: Wed Apr 25 07:06:42 2007
New Revision: 532360

URL: http://svn.apache.org/viewvc?view=rev&rev=532360
Log:
added support for a fixed size LRU based memory Idempotent Consumer

Added:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/util/LRUCache.java   (with props)
Modified:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/idempotent/MemoryMessageIdRepository.java
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/builder/RouteBuilderTest.java
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/IdempotentConsumerTest.java

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/idempotent/MemoryMessageIdRepository.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/idempotent/MemoryMessageIdRepository.java?view=diff&rev=532360&r1=532359&r2=532360
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/idempotent/MemoryMessageIdRepository.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/idempotent/MemoryMessageIdRepository.java Wed Apr 25 07:06:42 2007
@@ -17,43 +17,59 @@
  */
 package org.apache.camel.processor.idempotent;
 
+import org.apache.camel.util.LRUCache;
+
 import java.util.Set;
 import java.util.HashSet;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
 
 /**
- * A simple memory implementation of {@link MessageIdRepository}; though warning this could use up lots of RAM!
+ * A memory based implementation of {@link MessageIdRepository}.
+ * Care should be taken to use a suitable underlying {@link Map} to avoid this class being a memory leak
  *
  * @version $Revision: 1.1 $
  */
 public class MemoryMessageIdRepository implements MessageIdRepository {
-    private Set set;
+    private Map cache;
 
     /**
-     * Creates a new MemoryMessageIdRepository with a memory based respository
+     * Creates a new MemoryMessageIdRepository with a memory based respository. <b>Warning</b> this
+     * method should only really be used for testing as it will involve keeping all message IDs in RAM.
      */
     public static MessageIdRepository memoryMessageIdRepository() {
-        return memoryMessageIdRepository(new HashSet());
+        return memoryMessageIdRepository(new HashMap());
+    }
+
+    /**
+     * Creates a new MemoryMessageIdRepository with a memory based respository. <b>Warning</b> this
+     * method should only really be used for testing as it will involve keeping all message IDs in RAM.
+     */
+    public static MessageIdRepository memoryMessageIdRepository(int cacheSize) {
+        return memoryMessageIdRepository(new LRUCache(cacheSize));
     }
 
     /**
-     * Creates a new MemoryMessageIdRepository using the given {@link Set} to use to store the
-     * processed Message ID objects
+     * Creates a new MemoryMessageIdRepository using the given {@link Map} to use to store the
+     * processed Message ID objects. Warning be cafeful of the implementation of Map you use as
+     * if you are not careful it could be a memory leak.
      */
-    public static MessageIdRepository memoryMessageIdRepository(Set set) {
-        return new MemoryMessageIdRepository(set);
+    public static MessageIdRepository memoryMessageIdRepository(Map cache) {
+        return new MemoryMessageIdRepository(cache);
     }
 
-    public MemoryMessageIdRepository(Set set) {
-        this.set = set;
+    public MemoryMessageIdRepository(Map set) {
+        this.cache = set;
     }
 
     public boolean contains(String messageId) {
-        synchronized (set) {
-            if (set.contains(messageId)) {
+        synchronized (cache) {
+            if (cache.containsKey(messageId)) {
                 return true;
             }
             else {
-                set.add(messageId);
+                cache.put(messageId, messageId);
                 return false;
             }
         }

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/util/LRUCache.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/util/LRUCache.java?view=auto&rev=532360
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/util/LRUCache.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/util/LRUCache.java Wed Apr 25 07:06:42 2007
@@ -0,0 +1,63 @@
+/**
+ *
+ * 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;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * A Least Recently Used Cache
+ *
+ * @version $Revision: 1.1 $
+ */
+public class LRUCache<K, V> extends LinkedHashMap<K, V> {
+    private static final long serialVersionUID = -342098639681884413L;
+    private int maxCacheSize = 10000;
+
+    public LRUCache(int maximumCacheSize) {
+        this(maximumCacheSize, maximumCacheSize, 0.75f, true);
+    }
+
+    /**
+     * Constructs an empty <tt>LRUCache</tt> instance with the
+     * specified initial capacity, maximumCacheSize,load factor and ordering mode.
+     *
+     * @param initialCapacity  the initial capacity.
+     * @param maximumCacheSize
+     * @param loadFactor       the load factor.
+     * @param accessOrder      the ordering mode - <tt>true</tt> for
+     *                         access-order, <tt>false</tt> for insertion-order.
+     * @throws IllegalArgumentException if the initial capacity is negative
+     *                                  or the load factor is nonpositive.
+     */
+    public LRUCache(int initialCapacity, int maximumCacheSize, float loadFactor, boolean accessOrder) {
+        super(initialCapacity, loadFactor, accessOrder);
+        this.maxCacheSize = maximumCacheSize;
+    }
+
+    /**
+     * @return Returns the maxCacheSize.
+     */
+    public int getMaxCacheSize() {
+        return maxCacheSize;
+    }
+
+    protected boolean removeEldestEntry(Map.Entry entry) {
+        return size() > maxCacheSize;
+    }
+}
\ No newline at end of file

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/util/LRUCache.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/builder/RouteBuilderTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/builder/RouteBuilderTest.java?view=diff&rev=532360&r1=532359&r2=532360
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/builder/RouteBuilderTest.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/builder/RouteBuilderTest.java Wed Apr 25 07:06:42 2007
@@ -373,7 +373,9 @@
         // START SNIPPET: idempotent
         RouteBuilder<Exchange> builder = new RouteBuilder<Exchange>() {
             public void configure() {
-                from("queue:a").idempotentConsumer(header("myMessageId"), memoryMessageIdRepository()).to("queue:b");
+                from("queue:a").idempotentConsumer(
+                        header("myMessageId"), memoryMessageIdRepository(200)
+                ).to("queue:b");
             }
         };
         // END SNIPPET: idempotent

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/IdempotentConsumerTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/IdempotentConsumerTest.java?view=diff&rev=532360&r1=532359&r2=532360
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/IdempotentConsumerTest.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/IdempotentConsumerTest.java Wed Apr 25 07:06:42 2007
@@ -78,7 +78,9 @@
     protected RouteBuilder createRouteBuilder() {
         return new RouteBuilder() {
             public void configure() {
-                from("direct:start").idempotentConsumer(header("messageId"), memoryMessageIdRepository()).to("mock:result");
+                from("direct:start").idempotentConsumer(
+                        header("messageId"), memoryMessageIdRepository(200)
+                ).to("mock:result");
             }
         };
     }