You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@marmotta.apache.org by ss...@apache.org on 2013/12/17 15:47:34 UTC

[1/7] git commit: towards MARMOTTA-388: changed KiWiCacheManager infrastructure

Updated Branches:
  refs/heads/develop 593ea4fa1 -> 92205ec47


towards MARMOTTA-388: changed KiWiCacheManager infrastructure


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

Branch: refs/heads/develop
Commit: c0c795b9a4cfac78f6cfa12f09ba89eea7aef0ff
Parents: e7b3cca
Author: Sebastian Schaffert <ss...@apache.org>
Authored: Mon Dec 16 17:55:59 2013 +0100
Committer: Sebastian Schaffert <ss...@apache.org>
Committed: Mon Dec 16 17:55:59 2013 +0100

----------------------------------------------------------------------
 .../kiwi/loader/generic/KiWiHandler.java        |   6 +-
 libraries/kiwi/kiwi-triplestore/pom.xml         |   6 +-
 .../marmotta/kiwi/caching/KiWiCacheManager.java | 162 ++++++++++++--
 .../marmotta/kiwi/caching/KiWiQueryCache.java   | 222 -------------------
 .../org/apache/marmotta/kiwi/caching/README.txt |   5 -
 .../kiwi/persistence/KiWiConnection.java        |  89 ++++----
 .../kiwi/persistence/KiWiPersistence.java       |   2 +-
 .../kiwi/persistence/KiWiTripleRegistry.java    | 147 ------------
 .../registry/CacheTripleRegistry.java           |  80 +++++++
 .../persistence/registry/DBTripleRegistry.java  | 152 +++++++++++++
 .../registry/KiWiTripleRegistry.java            |  61 +++++
 .../marmotta/kiwi/sail/KiWiValueFactory.java    |  86 +------
 .../kiwi/persistence/h2/statements.properties   |   2 +-
 .../persistence/mysql/statements.properties     |   2 +-
 .../persistence/pgsql/statements.properties     |   2 +-
 parent/pom.xml                                  |  17 ++
 16 files changed, 508 insertions(+), 533 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java b/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java
index 8eca550..244d660 100644
--- a/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java
+++ b/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java
@@ -12,7 +12,7 @@ import org.apache.marmotta.commons.util.DateUtils;
 import org.apache.marmotta.kiwi.loader.KiWiLoaderConfiguration;
 import org.apache.marmotta.kiwi.model.rdf.*;
 import org.apache.marmotta.kiwi.persistence.KiWiConnection;
-import org.apache.marmotta.kiwi.persistence.KiWiTripleRegistry;
+import org.apache.marmotta.kiwi.persistence.registry.DBTripleRegistry;
 import org.apache.marmotta.kiwi.sail.KiWiStore;
 import org.openrdf.model.*;
 import org.openrdf.model.impl.URIImpl;
@@ -64,7 +64,7 @@ public class KiWiHandler implements RDFHandler {
     private Statistics statistics;
 
     // only used when statement existance check is enabled
-    protected KiWiTripleRegistry registry;
+    protected DBTripleRegistry registry;
 
 
     protected Date importDate;
@@ -114,7 +114,7 @@ public class KiWiHandler implements RDFHandler {
 
 
         if(config.isStatementExistanceCheck()) {
-            registry = new KiWiTripleRegistry(store);
+            registry = new DBTripleRegistry(store);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/pom.xml
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/pom.xml b/libraries/kiwi/kiwi-triplestore/pom.xml
index e815cce..58a7b72 100644
--- a/libraries/kiwi/kiwi-triplestore/pom.xml
+++ b/libraries/kiwi/kiwi-triplestore/pom.xml
@@ -59,10 +59,10 @@
         </dependency>
 
 
-        <!-- Caching (transactional) -->
+        <!-- Caching  -->
         <dependency>
-            <groupId>net.sf.ehcache</groupId>
-            <artifactId>ehcache-core</artifactId>
+            <groupId>org.infinispan</groupId>
+            <artifactId>infinispan-core</artifactId>
         </dependency>
 
 

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
index 3ea69f8..1c444a4 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
@@ -17,8 +17,21 @@
  */
 package org.apache.marmotta.kiwi.caching;
 
-import net.sf.ehcache.Cache;
-import net.sf.ehcache.CacheManager;
+import org.apache.marmotta.kiwi.config.KiWiConfiguration;
+import org.infinispan.Cache;
+import org.infinispan.configuration.cache.CacheMode;
+import org.infinispan.configuration.cache.Configuration;
+import org.infinispan.configuration.cache.ConfigurationBuilder;
+import org.infinispan.configuration.global.GlobalConfiguration;
+import org.infinispan.configuration.global.GlobalConfigurationBuilder;
+import org.infinispan.distribution.ch.SyncConsistentHashFactory;
+import org.infinispan.eviction.EvictionStrategy;
+import org.infinispan.manager.DefaultCacheManager;
+import org.infinispan.manager.EmbeddedCacheManager;
+
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 /**
  * A class for managing the different caches that are used by the triple store.
@@ -28,16 +41,50 @@ import net.sf.ehcache.CacheManager;
 public class KiWiCacheManager {
 
 
-    private CacheManager cacheManager;
+    public static final String NODE_CACHE = "node-cache";
+    public static final String TRIPLE_CACHE = "triple-cache";
+    public static final String URI_CACHE = "uri-cache";
+    public static final String BNODE_CACHE = "bnode-cache";
+    public static final String LITERAL_CACHE = "literal-cache";
+    public static final String NAMESPACE_URI_CACHE = "namespace-uri-cache";
+    public static final String NAMESPACE_PREFIX_CACHE = "namespace-prefix-cache";
+    public static final String LOADER_CACHE = "loader-cache";
+    private EmbeddedCacheManager cacheManager;
 
-    private KiWiQueryCache queryCache;
+    private GlobalConfiguration globalConfiguration;
 
-    public KiWiCacheManager(String name) {
-        cacheManager = CacheManager.newInstance(KiWiCacheManager.class.getResource("/ehcache-kiwi.xml"));
-        if(name != null) {
-            cacheManager.setName(name);
-        }
-        queryCache = new KiWiQueryCache(cacheManager.getCache("query-cache"));
+    // default configuration: distributed cache, 100000 entries, 300 seconds expiration, 60 seconds idle
+    private Configuration defaultConfiguration;
+
+    public KiWiCacheManager(KiWiConfiguration config) {
+        globalConfiguration = new GlobalConfigurationBuilder()
+                .transport()
+                    .defaultTransport()
+                    .clusterName(config.getName())
+                    .machineId("instance-" + config.getDatacenterId())
+                    .addProperty("configurationFile", "jgroups-udp.xml")
+                .build();
+
+
+        defaultConfiguration = new ConfigurationBuilder()
+                .clustering()
+                    .cacheMode(CacheMode.DIST_SYNC)
+                    .sync()
+                    .l1()
+                        .lifespan(25, TimeUnit.SECONDS)
+                    .hash()
+                        .numOwners(2)
+                        .numSegments(100)
+                        .consistentHashFactory(new SyncConsistentHashFactory())
+                .eviction()
+                    .strategy(EvictionStrategy.LIRS)
+                    .maxEntries(100000)
+                .expiration()
+                    .lifespan(5, TimeUnit.MINUTES)
+                    .maxIdle(1, TimeUnit.MINUTES)
+                .build();
+
+        cacheManager = new DefaultCacheManager(globalConfiguration, defaultConfiguration, true);
     }
 
     /**
@@ -47,7 +94,15 @@ public class KiWiCacheManager {
      * @return an EHCache Cache instance containing the node id -> node mappings
      */
     public Cache getNodeCache() {
-        return cacheManager.getCache("node-cache");
+        if(!cacheManager.cacheExists(NODE_CACHE)) {
+            Configuration nodeConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .eviction()
+                        .maxEntries(500000)
+                    .build();
+            cacheManager.defineConfiguration(NODE_CACHE, nodeConfiguration);
+        }
+
+        return cacheManager.getCache(NODE_CACHE);
     }
 
     /**
@@ -57,7 +112,15 @@ public class KiWiCacheManager {
      * @return
      */
     public Cache getTripleCache() {
-        return cacheManager.getCache("triple-cache");
+        if(!cacheManager.cacheExists(TRIPLE_CACHE)) {
+            Configuration tripleConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .expiration()
+                        .lifespan(60, TimeUnit.SECONDS)
+                        .maxIdle(30, TimeUnit.SECONDS)
+                    .build();
+            cacheManager.defineConfiguration(TRIPLE_CACHE, tripleConfiguration);
+        }
+        return cacheManager.getCache(TRIPLE_CACHE);
     }
 
 
@@ -68,7 +131,14 @@ public class KiWiCacheManager {
      * @return
      */
     public Cache getUriCache() {
-        return cacheManager.getCache("uri-cache");
+        if(!cacheManager.cacheExists(URI_CACHE)) {
+            Configuration uriConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .eviction()
+                        .maxEntries(50000)
+                    .build();
+            cacheManager.defineConfiguration(URI_CACHE, uriConfiguration);
+        }
+        return cacheManager.getCache(URI_CACHE);
     }
 
 
@@ -79,7 +149,14 @@ public class KiWiCacheManager {
      * @return
      */
     public Cache getBNodeCache() {
-        return cacheManager.getCache("bnode-cache");
+        if(!cacheManager.cacheExists(BNODE_CACHE)) {
+            Configuration bnodeConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .eviction()
+                        .maxEntries(5000)
+                    .build();
+            cacheManager.defineConfiguration(BNODE_CACHE, bnodeConfiguration);
+        }
+        return cacheManager.getCache(BNODE_CACHE);
     }
 
     /**
@@ -90,7 +167,14 @@ public class KiWiCacheManager {
      * @return
      */
     public Cache getLiteralCache() {
-        return cacheManager.getCache("literal-cache");
+        if(!cacheManager.cacheExists(LITERAL_CACHE)) {
+            Configuration literalConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .eviction()
+                        .maxEntries(10000)
+                    .build();
+            cacheManager.defineConfiguration(LITERAL_CACHE, literalConfiguration);
+        }
+        return cacheManager.getCache(LITERAL_CACHE);
     }
 
 
@@ -99,7 +183,16 @@ public class KiWiCacheManager {
      * @return
      */
     public Cache getNamespaceUriCache() {
-        return cacheManager.getCache("namespace-uri-cache");
+        if(!cacheManager.cacheExists(NAMESPACE_URI_CACHE)) {
+            Configuration nsuriConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .eviction()
+                        .maxEntries(1000)
+                    .expiration()
+                        .lifespan(1, TimeUnit.HOURS)
+                    .build();
+            cacheManager.defineConfiguration(NAMESPACE_URI_CACHE, nsuriConfiguration);
+        }
+        return cacheManager.getCache(NAMESPACE_URI_CACHE);
     }
 
     /**
@@ -107,7 +200,16 @@ public class KiWiCacheManager {
      * @return
      */
     public Cache getNamespacePrefixCache() {
-        return cacheManager.getCache("namespace-prefix-cache");
+        if(!cacheManager.cacheExists(NAMESPACE_PREFIX_CACHE)) {
+            Configuration nsprefixConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .eviction()
+                        .maxEntries(1000)
+                    .expiration()
+                        .lifespan(1, TimeUnit.HOURS)
+                    .build();
+            cacheManager.defineConfiguration(NAMESPACE_PREFIX_CACHE, nsprefixConfiguration);
+        }
+        return cacheManager.getCache(NAMESPACE_PREFIX_CACHE);
     }
 
 
@@ -116,7 +218,17 @@ public class KiWiCacheManager {
      * @return
      */
     public Cache getLoaderCache() {
-        return cacheManager.getCache("loader-cache");
+        if(!cacheManager.cacheExists(LOADER_CACHE)) {
+            Configuration loaderConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .eviction()
+                        .maxEntries(100000)
+                    .expiration()
+                        .lifespan(10, TimeUnit.MINUTES)
+                        .maxIdle(30, TimeUnit.SECONDS)
+                    .build();
+            cacheManager.defineConfiguration(LOADER_CACHE, loaderConfiguration);
+        }
+        return cacheManager.getCache(LOADER_CACHE);
     }
 
 
@@ -128,9 +240,6 @@ public class KiWiCacheManager {
      * @return
      */
     public synchronized Cache getCacheByName(String name) {
-        if(!cacheManager.cacheExists(name)) {
-             cacheManager.addCache(name);
-        }
         return cacheManager.getCache(name);
 
     }
@@ -140,13 +249,18 @@ public class KiWiCacheManager {
      * Clear all caches managed by this cache manager.
      */
     public void clear() {
-        cacheManager.clearAll();
-    }
+        Set<String> set =  cacheManager.getCacheNames();
+        Iterator<String> iterator =  set.iterator();
+        while(iterator.hasNext()){
+            String cacheName = iterator.next();
+            Cache<String,Object> cache = cacheManager.getCache(cacheName);
+            cache.clear();
+        }      }
 
     /**
      * Shutdown this cache manager instance. Will shutdown the underlying EHCache cache manager.
      */
     public void shutdown() {
-        cacheManager.shutdown();
+        cacheManager.stop();
     }
 }

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiQueryCache.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiQueryCache.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiQueryCache.java
deleted file mode 100644
index cf9b457..0000000
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiQueryCache.java
+++ /dev/null
@@ -1,222 +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.marmotta.kiwi.caching;
-
-import com.google.common.collect.ImmutableList;
-import net.sf.ehcache.Ehcache;
-import net.sf.ehcache.Element;
-
-import org.apache.marmotta.commons.sesame.tripletable.IntArray;
-import org.apache.marmotta.kiwi.model.rdf.KiWiNode;
-import org.apache.marmotta.kiwi.model.rdf.KiWiResource;
-import org.apache.marmotta.kiwi.model.rdf.KiWiTriple;
-import org.apache.marmotta.kiwi.model.rdf.KiWiUriResource;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.nio.IntBuffer;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Add file description here!
- * <p/>
- * User: sschaffe
- */
-public class KiWiQueryCache  {
-
-    private static Logger log = LoggerFactory.getLogger(KiWiQueryCache.class);
-
-    private Ehcache queryCache;
-
-
-    public KiWiQueryCache(Ehcache queryCache) {
-        this.queryCache = queryCache;
-        log.debug("Building up new {}", KiWiQueryCache.class.getSimpleName());
-    }
-
-
-    /**
-     * Look up a triple query in the query cache. Returns the result set if the query is found in the cache, returns
-     * null if the query is not found.
-     *
-     * @param subject  the subject of the triples to list or null for wildcard
-     * @param property the property of the triples to list or null for wildcard
-     * @param object   the object of the triples to list or null for wildcard
-     * @param context  the context/knowledge space of the triples to list or null for all spaces
-     * @param inferred if true, inferred triples are included in the result; if false not
-     * @return the result set if the query is found in the cache, returns null if the query is not found
-     */
-    @SuppressWarnings("unchecked")
-	public List<KiWiTriple> listTriples(KiWiResource subject, KiWiUriResource property, KiWiNode object, KiWiUriResource context, boolean inferred) {
-        IntArray key = createCacheKey(subject,property,object,context,inferred);
-        if(queryCache.get(key) != null) return (List<KiWiTriple>)queryCache.get(key).getObjectValue();
-        else
-            return null;
-    }
-
-
-    /**
-     * Cache the result of a triple query in the query cache.
-     *
-     * @param subject  the subject of the triples to list or null for wildcard
-     * @param property the property of the triples to list or null for wildcard
-     * @param object   the object of the triples to list or null for wildcard
-     * @param context  the context/knowledge space of the triples to list or null for all spaces
-     * @param inferred if true, inferred triples are included in the result; if false not
-     * @param result   the result of the triple query to cache
-     */
-    public void cacheTriples(KiWiResource subject, KiWiUriResource property, KiWiNode object, KiWiResource context, boolean inferred, List<KiWiTriple> result) {
-
-        // cache the query result
-        IntArray key = createCacheKey(subject,property,object,context,inferred);
-        queryCache.put(new Element(key,result));
-
-        // cache the nodes of the triples and the triples themselves
-        Set<KiWiNode> nodes = new HashSet<KiWiNode>();
-        for(KiWiTriple triple : result) {
-            Collections.addAll(nodes, new KiWiNode[]{triple.getSubject(), triple.getObject(), triple.getPredicate(), triple.getContext()});
-            queryCache.put(new Element(createCacheKey(triple.getSubject(),triple.getPredicate(),triple.getObject(),triple.getContext(),triple.isInferred()), ImmutableList.of(triple)));
-        }
-
-        // special optimisation: when only the subject (and optionally context) is given, we also fill the caches for
-        // all property values
-        if(subject != null && property == null && object == null) {
-            HashMap<KiWiUriResource,List<KiWiTriple>> properties = new HashMap<KiWiUriResource, List<KiWiTriple>>();
-            for(KiWiTriple triple : result) {
-                List<KiWiTriple> values = properties.get(triple.getPredicate());
-                if(values == null) {
-                    values = new LinkedList<KiWiTriple>();
-                    properties.put(triple.getPredicate(),values);
-                }
-                values.add(triple);
-            }
-            for(Map.Entry<KiWiUriResource,List<KiWiTriple>> entry : properties.entrySet()) {
-                IntArray key2 = createCacheKey(subject,entry.getKey(),null,context,inferred);
-                queryCache.put(new Element(key2,entry.getValue()));
-            }
-        }
-
-
-    }
-
-
-    /**
-     * Add the triple given as argument to the triple cache. This method ensures that all other cache entries that
-     * are affected by the addition of this triple will be expired.
-     *
-     * @param triple
-     */
-    public void cacheTriple(KiWiTriple triple) {
-        tripleUpdated(triple);
-        cacheTriples(triple.getSubject(), triple.getPredicate(), triple.getObject(), triple.getContext(), triple.isInferred(), Collections.singletonList(triple));
-    }
-
-
-    /**
-     * Remove the triple given as argument from the triple cache. This method ensures that all other cache entries
-     * that are affected by the removal of this triple will be expired.
-     * @param triple
-     */
-    public void removeTriple(KiWiTriple triple) {
-        tripleUpdated(triple);
-    }
-
-
-    /**
-     * Clear all contents of the query cache.
-     */
-    public void clearAll() {
-        queryCache.removeAll();
-    }
-
-
-    /**
-     * Notify the cache that the triple passed as argument has been updated and that all cache entries affected by
-     * the triple update need to be cleared.
-     *
-     * @param triple
-     */
-    public void tripleUpdated(KiWiTriple triple) {
-        queryCache.remove(createCacheKey(null,null,null,null,false));
-        queryCache.remove(createCacheKey(null,null,null,null,true));
-
-        // remove all possible combinations of this triple as they may appear in the cache
-        queryCache.remove(createCacheKey(triple.getSubject(),null,null,null,false));
-        queryCache.remove(createCacheKey(triple.getSubject(),null,null,null,true));
-        queryCache.remove(createCacheKey(null,triple.getPredicate(),null,null,false));
-        queryCache.remove(createCacheKey(null,triple.getPredicate(),null,null,true));
-        queryCache.remove(createCacheKey(null,null,triple.getObject(),null,false));
-        queryCache.remove(createCacheKey(null,null,triple.getObject(),null,true));
-        queryCache.remove(createCacheKey(null,null,null,triple.getContext(),false));
-        queryCache.remove(createCacheKey(null,null,null,triple.getContext(),true));
-
-        queryCache.remove(createCacheKey(triple.getSubject(),triple.getPredicate(),null,null,false));
-        queryCache.remove(createCacheKey(triple.getSubject(),triple.getPredicate(),null,null,true));
-        queryCache.remove(createCacheKey(triple.getSubject(),null,triple.getObject(),null,false));
-        queryCache.remove(createCacheKey(triple.getSubject(),null,triple.getObject(),null,true));
-        queryCache.remove(createCacheKey(triple.getSubject(),null,null,triple.getContext(),false));
-        queryCache.remove(createCacheKey(triple.getSubject(),null,null,triple.getContext(),true));
-        queryCache.remove(createCacheKey(null,triple.getPredicate(),triple.getObject(),null,false));
-        queryCache.remove(createCacheKey(null,triple.getPredicate(),triple.getObject(),null,true));
-        queryCache.remove(createCacheKey(null,triple.getPredicate(),null,triple.getContext(),false));
-        queryCache.remove(createCacheKey(null,triple.getPredicate(),null,triple.getContext(),true));
-        queryCache.remove(createCacheKey(null,null,triple.getObject(),triple.getContext(),false));
-        queryCache.remove(createCacheKey(null,null,triple.getObject(),triple.getContext(),true));
-
-
-        queryCache.remove(createCacheKey(triple.getSubject(),triple.getPredicate(),triple.getObject(),null,false));
-        queryCache.remove(createCacheKey(triple.getSubject(),triple.getPredicate(),triple.getObject(),null,true));
-        queryCache.remove(createCacheKey(triple.getSubject(),triple.getPredicate(),null,triple.getContext(),false));
-        queryCache.remove(createCacheKey(triple.getSubject(),triple.getPredicate(),null,triple.getContext(),true));
-        queryCache.remove(createCacheKey(triple.getSubject(),null,triple.getObject(),triple.getContext(),false));
-        queryCache.remove(createCacheKey(triple.getSubject(),null,triple.getObject(),triple.getContext(),true));
-        queryCache.remove(createCacheKey(null,triple.getPredicate(),triple.getObject(),triple.getContext(),false));
-        queryCache.remove(createCacheKey(null,triple.getPredicate(),triple.getObject(),triple.getContext(),true));
-
-        queryCache.remove(createCacheKey(triple.getSubject(),triple.getPredicate(),triple.getObject(),triple.getContext(),false));
-        queryCache.remove(createCacheKey(triple.getSubject(),triple.getPredicate(),triple.getObject(),triple.getContext(),true));
-    }
-
-
-    private static IntArray createCacheKey(KiWiResource subject, KiWiUriResource property, KiWiNode object, KiWiResource context, boolean inferred){
-
-        // the cache key is generated by appending the bytes of the hashcodes of subject, property, object, context and inferred and
-        // storing them as a BigInteger; generating the cache key should thus be very efficient
-
-        int s = subject != null ? subject.hashCode() : Integer.MIN_VALUE;
-        int p = property != null ? property.hashCode() : Integer.MIN_VALUE;
-        int o = object != null ? object.hashCode() : Integer.MIN_VALUE;
-        int c = context != null ? context.hashCode() : Integer.MIN_VALUE;
-
-        IntBuffer bb = IntBuffer.allocate(5);
-        bb.put(s);
-        bb.put(p);
-        bb.put(o);
-        bb.put(c);
-        bb.put( (byte) (inferred ? 1 : 0) );
-
-        return new IntArray(bb.array());
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/README.txt
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/README.txt b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/README.txt
deleted file mode 100644
index 4a9b4d2..0000000
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/README.txt
+++ /dev/null
@@ -1,5 +0,0 @@
-Caching is used for the following tasks:
-- cache query results (i.e. calls to listTriples); this caching should address cursor/iterator operations for large
-  data sets
-- cache individual triples, resources, namespaces for get/create methods
-- cache individual triples, resources, namespaces for id-based lookups
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiConnection.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiConnection.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiConnection.java
index 36dd591..778d8fe 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiConnection.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiConnection.java
@@ -19,8 +19,6 @@ package org.apache.marmotta.kiwi.persistence;
 
 import com.google.common.base.Preconditions;
 import info.aduna.iteration.*;
-import net.sf.ehcache.Cache;
-import net.sf.ehcache.Element;
 import org.apache.commons.lang3.math.NumberUtils;
 import org.apache.marmotta.commons.sesame.model.LiteralCommons;
 import org.apache.marmotta.commons.sesame.model.Namespaces;
@@ -32,6 +30,7 @@ import org.apache.marmotta.kiwi.exception.ResultInterruptedException;
 import org.apache.marmotta.kiwi.model.rdf.*;
 import org.apache.marmotta.kiwi.persistence.util.ResultSetIteration;
 import org.apache.marmotta.kiwi.persistence.util.ResultTransformerFunction;
+import org.infinispan.Cache;
 import org.openrdf.model.Literal;
 import org.openrdf.model.Resource;
 import org.openrdf.model.Statement;
@@ -70,40 +69,40 @@ public class KiWiConnection {
     /**
      * Cache nodes by database ID
      */
-    private Cache nodeCache;
+    private Cache<Long,KiWiNode> nodeCache;
 
     /**
      * Cache triples by database ID
      */
-    private Cache tripleCache;
+    private Cache<Long,KiWiTriple> tripleCache;
 
 
     /**
      * Cache URI resources by uri
      */
-    private Cache uriCache;
+    private Cache<String,KiWiUriResource> uriCache;
 
 
     /**
      * Cache BNodes by BNode ID
      */
-    private Cache bnodeCache;
+    private Cache<String,KiWiAnonResource> bnodeCache;
 
     /**
      * Cache literals by literal cache key (LiteralCommons#createCacheKey(String,Locale,URI))
      */
-    private Cache literalCache;
+    private Cache<String,KiWiLiteral> literalCache;
 
 
     /**
      * Look up namespaces by URI
      */
-    private Cache namespaceUriCache;
+    private Cache<String,KiWiNamespace> namespaceUriCache;
 
     /**
      * Look up namespaces by prefix
      */
-    private Cache namespacePrefixCache;
+    private Cache<String,KiWiNamespace> namespacePrefixCache;
 
     /**
      * Cache instances of locales for language tags
@@ -227,9 +226,9 @@ public class KiWiConnection {
      * @throws SQLException
      */
     public KiWiNamespace loadNamespaceByPrefix(String prefix) throws SQLException {
-        Element element = namespacePrefixCache.get(prefix);
+        KiWiNamespace element = namespacePrefixCache.get(prefix);
         if(element != null) {
-            return (KiWiNamespace)element.getObjectValue();
+            return element;
         }
 
         requireJDBCConnection();
@@ -263,9 +262,9 @@ public class KiWiConnection {
      * @throws SQLException
      */
     public KiWiNamespace loadNamespaceByUri(String uri) throws SQLException {
-        Element element = namespaceUriCache.get(uri);
+        KiWiNamespace element = namespaceUriCache.get(uri);
         if(element != null) {
-            return (KiWiNamespace)element.getObjectValue();
+            return element;
         }
 
         requireJDBCConnection();
@@ -315,8 +314,8 @@ public class KiWiConnection {
 
         insertNamespace.executeUpdate();
 
-        namespacePrefixCache.put(new Element(namespace.getPrefix(),namespace));
-        namespaceUriCache.put(new Element(namespace.getUri(),namespace));
+        namespacePrefixCache.put(namespace.getPrefix(), namespace);
+        namespaceUriCache.put(namespace.getUri(),namespace);
     }
 
     /**
@@ -417,9 +416,9 @@ public class KiWiConnection {
     public KiWiNode loadNodeById(Long id) throws SQLException {
 
         // look in cache
-        Element element = nodeCache.get(id);
+        KiWiNode element = nodeCache.get(id);
         if(element != null) {
-            return (KiWiNode)element.getObjectValue();
+            return element;
         }
 
         requireJDBCConnection();
@@ -449,9 +448,9 @@ public class KiWiConnection {
     public KiWiTriple loadTripleById(Long id) throws SQLException {
 
         // look in cache
-        Element element = tripleCache.get(id);
+        KiWiTriple element = tripleCache.get(id);
         if(element != null) {
-            return (KiWiTriple)element.getObjectValue();
+            return element;
         }
 
         requireJDBCConnection();
@@ -491,9 +490,9 @@ public class KiWiConnection {
         Preconditions.checkNotNull(uri);
 
         // look in cache
-        Element element = uriCache.get(uri);
+        KiWiUriResource element = uriCache.get(uri);
         if(element != null) {
-            return (KiWiUriResource)element.getObjectValue();
+            return element;
         }
 
         requireJDBCConnection();
@@ -537,9 +536,9 @@ public class KiWiConnection {
      */
     public KiWiAnonResource loadAnonResource(String id) throws SQLException {
         // look in cache
-        Element element = bnodeCache.get(id);
+        KiWiAnonResource element = bnodeCache.get(id);
         if(element != null) {
-            return (KiWiAnonResource)element.getObjectValue();
+            return element;
         }
 
         requireJDBCConnection();
@@ -587,9 +586,9 @@ public class KiWiConnection {
      */
     public KiWiLiteral loadLiteral(String value, String lang, KiWiUriResource ltype) throws SQLException {
         // look in cache
-        final Element element = literalCache.get(LiteralCommons.createCacheKey(value,getLocale(lang), ltype));
+        final KiWiLiteral element = literalCache.get(LiteralCommons.createCacheKey(value,getLocale(lang), ltype));
         if(element != null) {
-            return (KiWiLiteral)element.getObjectValue();
+            return element;
         }
 
         requireJDBCConnection();
@@ -652,9 +651,9 @@ public class KiWiConnection {
      */
     public KiWiDateLiteral loadLiteral(Date date) throws SQLException {
         // look in cache
-        Element element = literalCache.get(LiteralCommons.createCacheKey(DateUtils.getDateWithoutFraction(date),Namespaces.NS_XSD + "dateTime"));
+        KiWiLiteral element = literalCache.get(LiteralCommons.createCacheKey(DateUtils.getDateWithoutFraction(date),Namespaces.NS_XSD + "dateTime"));
         if(element != null) {
-            return (KiWiDateLiteral)element.getObjectValue();
+            return (KiWiDateLiteral)element;
         }
 
         requireJDBCConnection();
@@ -706,9 +705,9 @@ public class KiWiConnection {
      */
     public KiWiIntLiteral loadLiteral(long value) throws SQLException {
         // look in cache
-        Element element = literalCache.get(LiteralCommons.createCacheKey(Long.toString(value),null,Namespaces.NS_XSD + "integer"));
+        KiWiLiteral element = literalCache.get(LiteralCommons.createCacheKey(Long.toString(value),null,Namespaces.NS_XSD + "integer"));
         if(element != null) {
-            return (KiWiIntLiteral)element.getObjectValue();
+            return (KiWiIntLiteral)element;
         }
 
         requireJDBCConnection();
@@ -761,9 +760,9 @@ public class KiWiConnection {
      */
     public synchronized KiWiDoubleLiteral loadLiteral(double value) throws SQLException {
         // look in cache
-        Element element = literalCache.get(LiteralCommons.createCacheKey(Double.toString(value),null,Namespaces.NS_XSD + "double"));
+        KiWiLiteral element = literalCache.get(LiteralCommons.createCacheKey(Double.toString(value),null,Namespaces.NS_XSD + "double"));
         if(element != null) {
-            return (KiWiDoubleLiteral)element.getObjectValue();
+            return (KiWiDoubleLiteral)element;
         }
 
         requireJDBCConnection();
@@ -815,9 +814,9 @@ public class KiWiConnection {
      */
     public KiWiBooleanLiteral loadLiteral(boolean value) throws SQLException {
         // look in cache
-        Element element = literalCache.get(LiteralCommons.createCacheKey(Boolean.toString(value),null,Namespaces.NS_XSD + "boolean"));
+        KiWiLiteral element = literalCache.get(LiteralCommons.createCacheKey(Boolean.toString(value),null,Namespaces.NS_XSD + "boolean"));
         if(element != null) {
-            return (KiWiBooleanLiteral)element.getObjectValue();
+            return (KiWiBooleanLiteral)element;
         }
 
 
@@ -1539,8 +1538,8 @@ public class KiWiConnection {
         result.setId(row.getLong("id"));
         result.setCreated(new Date(row.getTimestamp("createdAt").getTime()));
 
-        namespacePrefixCache.put(new Element(result.getPrefix(),result));
-        namespaceUriCache.put(new Element(result.getUri(),result));
+        namespacePrefixCache.put(result.getPrefix(),result);
+        namespaceUriCache.put(result.getUri(),result);
 
         return result;
     }
@@ -1555,11 +1554,11 @@ public class KiWiConnection {
 
         long id = row.getLong("id");
 
-        Element cached = nodeCache.get(id);
+        KiWiNode cached = nodeCache.get(id);
 
         // lookup element in cache first, so we can avoid reconstructing it if it is already there
         if(cached != null) {
-            return (KiWiNode)cached.getObjectValue();
+            return cached;
         }
 
         String ntype = row.getString("ntype");
@@ -1658,11 +1657,11 @@ public class KiWiConnection {
 
         Long id = row.getLong("id");
 
-        Element cached = tripleCache.get(id);
+        KiWiTriple cached = tripleCache.get(id);
 
         // lookup element in cache first, so we can avoid reconstructing it if it is already there
         if(cached != null) {
-            return (KiWiTriple)cached.getObjectValue();
+            return cached;
         }
 
         KiWiTriple result = new KiWiTriple();
@@ -1686,7 +1685,7 @@ public class KiWiConnection {
             // (see http://stackoverflow.com/questions/782823/handling-datetime-values-0000-00-00-000000-in-jdbc)
         }
 
-        tripleCache.put(new Element(id,result));
+        tripleCache.put(id,result);
 
         return result;
     }
@@ -1769,20 +1768,20 @@ public class KiWiConnection {
 
     private void cacheNode(KiWiNode node) {
         if(node.getId() >= 0) {
-            nodeCache.put(new Element(node.getId(), node));
+            nodeCache.put(node.getId(), node);
         }
         if(node instanceof KiWiUriResource) {
-            uriCache.put(new Element(node.stringValue(), node));
+            uriCache.put(node.stringValue(), (KiWiUriResource) node);
         } else if(node instanceof KiWiAnonResource) {
-            bnodeCache.put(new Element(node.stringValue(),node));
+            bnodeCache.put(node.stringValue(), (KiWiAnonResource) node);
         } else if(node instanceof KiWiLiteral) {
-            literalCache.put(new Element(LiteralCommons.createCacheKey((Literal) node),node));
+            literalCache.put(LiteralCommons.createCacheKey((Literal) node), (KiWiLiteral) node);
         }
     }
 
     private void cacheTriple(KiWiTriple triple) {
         if(triple.getId() >= 0) {
-            tripleCache.put(new Element(triple.getId(),triple));
+            tripleCache.put(triple.getId(),triple);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiPersistence.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiPersistence.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiPersistence.java
index 360f4b7..aa70ad0 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiPersistence.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiPersistence.java
@@ -128,7 +128,7 @@ public class KiWiPersistence {
 
 
     private void initCachePool() {
-        cacheManager = new KiWiCacheManager(configuration.getName());
+        cacheManager = new KiWiCacheManager(configuration);
     }
 
 

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiTripleRegistry.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiTripleRegistry.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiTripleRegistry.java
deleted file mode 100644
index dfdee6b..0000000
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/KiWiTripleRegistry.java
+++ /dev/null
@@ -1,147 +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.marmotta.kiwi.persistence;
-
-import org.apache.marmotta.commons.sesame.tripletable.IntArray;
-import org.apache.marmotta.kiwi.sail.KiWiStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-
-/**
- * This class is used for keeping triples that are used by several transactions in parallel in sync. It allows
- * a transaction that creates a triple to register this triple and make the triple ID available to other transactions
- * to avoid duplicates. When a transaction commits, it then releases all its triple registrations.
- * <p/>
- * The implementation is based on a very simple database table (REGISTRY). When a transaction creates a triple
- * with a new ID, it temporarily inserts a row mapping the (subject,predicate,object,context) -> triple ID. Other
- * transactions trying to create the same triple can then first lookup this ID. If they do so successfully, they will
- * also insert a row to the registry.
- *
- * @author Sebastian Schaffert (sschaffert@apache.org)
- */
-public class KiWiTripleRegistry {
-
-    private static Logger log = LoggerFactory.getLogger(KiWiTripleRegistry.class);
-
-    private KiWiStore store;
-
-
-    public KiWiTripleRegistry(KiWiStore store) {
-        this.store = store;
-    }
-
-    public void registerKey(IntArray key, long transactionId, long tripleId) {
-        KiWiConnection con = aqcuireConnection();
-        try {
-            PreparedStatement stmt = con.getPreparedStatement("registry.register");
-            synchronized (stmt) {
-                stmt.setLong(1, key.longHashCode());
-                stmt.setLong(2, tripleId);
-                stmt.setLong(3, transactionId);
-                stmt.executeUpdate();
-            }
-        } catch (SQLException e) {
-            log.error("error registering key in temporary database table",e);
-        } finally {
-            releaseConnection(con);
-        }
-    }
-
-
-
-    public long lookupKey(IntArray key) {
-        KiWiConnection con = aqcuireConnection();
-        try {
-            PreparedStatement stmt = con.getPreparedStatement("registry.lookup");
-            synchronized (stmt) {
-                stmt.setLong(1, key.longHashCode());
-
-                try(ResultSet r = stmt.executeQuery()) {
-                    if(r.next()) {
-                        return r.getLong(1);
-                    }
-                }
-            }
-        } catch (SQLException e) {
-            log.error("error looking up key in temporary database table",e);
-        } finally {
-            releaseConnection(con);
-        }
-        return -1;
-    }
-
-
-    public void releaseTransaction(long transactionId) {
-        KiWiConnection con = aqcuireConnection();
-        try {
-            PreparedStatement stmt = con.getPreparedStatement("registry.release");
-            synchronized (stmt) {
-                stmt.setLong(1, transactionId);
-                stmt.executeUpdate();
-            }
-        } catch (SQLException e) {
-            log.error("error releasing key in temporary database table",e);
-        } finally {
-            releaseConnection(con);
-        }
-
-    }
-
-
-    public void deleteKey(long tripleId) {
-        KiWiConnection con = aqcuireConnection();
-        try {
-            PreparedStatement stmt = con.getPreparedStatement("registry.delete");
-            synchronized (stmt) {
-                stmt.setLong(1, tripleId);
-                stmt.executeUpdate();
-            }
-        } catch (SQLException e) {
-            log.error("error deleting key in temporary database table",e);
-        } finally {
-            releaseConnection(con);
-        }
-
-    }
-
-
-    protected KiWiConnection aqcuireConnection() {
-        try {
-            return store.getPersistence().getConnection();
-        } catch(SQLException ex) {
-            log.error("could not acquire database connection", ex);
-            throw new RuntimeException(ex);
-        }
-    }
-
-    protected void releaseConnection(KiWiConnection con) {
-        try {
-            con.getJDBCConnection().commit();
-            con.close();
-        } catch (SQLException ex) {
-            log.error("could not release database connection", ex);
-            throw new RuntimeException(ex);
-        }
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/CacheTripleRegistry.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/CacheTripleRegistry.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/CacheTripleRegistry.java
new file mode 100644
index 0000000..cb1896c
--- /dev/null
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/CacheTripleRegistry.java
@@ -0,0 +1,80 @@
+/*
+ * 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.marmotta.kiwi.persistence.registry;
+
+import org.apache.marmotta.commons.sesame.tripletable.IntArray;
+import org.infinispan.AdvancedCache;
+
+import java.util.Map;
+
+/**
+ * A triple registry implementation based on the Infinispan cache.
+ *
+ * @author Sebastian Schaffert (sschaffert@apache.org)
+ */
+public class CacheTripleRegistry implements KiWiTripleRegistry {
+
+
+    AdvancedCache<IntArray,Map<Long,Long>> cache;
+
+
+    /**
+     * Register a key/triple id pair in the triple registry for the given transaction ID.
+     *
+     * @param key           the key identifying the triple arguments (subject, object, predicate, context)
+     * @param transactionId the identifier of the transaction registering the triple id
+     * @param tripleId      the new triple identifier
+     */
+    @Override
+    public void registerKey(IntArray key, long transactionId, long tripleId) {
+
+    }
+
+    /**
+     * Check if another (or the same) transaction has already registered an ID for the triple with the
+     * given key. Returns -1 in case no other ID has been registered, or a value >0 otherwise.
+     *
+     * @param key the key identifying the triple arguments (subject, object, predicate, context)
+     * @return id of the triple or -1
+     */
+    @Override
+    public long lookupKey(IntArray key) {
+        return 0;
+    }
+
+    /**
+     * Free all registry entries claimed by the transaction with the given identifier. Should remove or
+     * expire all registry entries to avoid unnecessary storage consumption.
+     *
+     * @param transactionId the identifier of the transaction registering the triple id
+     */
+    @Override
+    public void releaseTransaction(long transactionId) {
+
+    }
+
+    /**
+     * Remove the key with the given key, e.g. when a statement is again deleted during a transaction.
+     *
+     * @param key the key identifying the triple arguments (subject, object, predicate, context)
+     */
+    @Override
+    public void deleteKey(IntArray key) {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/DBTripleRegistry.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/DBTripleRegistry.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/DBTripleRegistry.java
new file mode 100644
index 0000000..e5a5852
--- /dev/null
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/DBTripleRegistry.java
@@ -0,0 +1,152 @@
+/*
+ * 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.marmotta.kiwi.persistence.registry;
+
+import org.apache.marmotta.commons.sesame.tripletable.IntArray;
+import org.apache.marmotta.kiwi.persistence.KiWiConnection;
+import org.apache.marmotta.kiwi.sail.KiWiStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+/**
+ * This class is used for keeping triples that are used by several transactions in parallel in sync. It allows
+ * a transaction that creates a triple to register this triple and make the triple ID available to other transactions
+ * to avoid duplicates. When a transaction commits, it then releases all its triple registrations.
+ * <p/>
+ * The implementation is based on a very simple database table (REGISTRY). When a transaction creates a triple
+ * with a new ID, it temporarily inserts a row mapping the (subject,predicate,object,context) -> triple ID. Other
+ * transactions trying to create the same triple can then first lookup this ID. If they do so successfully, they will
+ * also insert a row to the registry.
+ *
+ * @author Sebastian Schaffert (sschaffert@apache.org)
+ */
+public class DBTripleRegistry implements KiWiTripleRegistry {
+
+    private static Logger log = LoggerFactory.getLogger(DBTripleRegistry.class);
+
+    private KiWiStore store;
+
+
+    public DBTripleRegistry(KiWiStore store) {
+        this.store = store;
+    }
+
+    @Override
+    public void registerKey(IntArray key, long transactionId, long tripleId) {
+        KiWiConnection con = aqcuireConnection();
+        try {
+            PreparedStatement stmt = con.getPreparedStatement("registry.register");
+            synchronized (stmt) {
+                stmt.setLong(1, key.longHashCode());
+                stmt.setLong(2, tripleId);
+                stmt.setLong(3, transactionId);
+                stmt.executeUpdate();
+            }
+        } catch (SQLException e) {
+            log.error("error registering key in temporary database table",e);
+        } finally {
+            releaseConnection(con);
+        }
+    }
+
+
+
+    @Override
+    public long lookupKey(IntArray key) {
+        KiWiConnection con = aqcuireConnection();
+        try {
+            PreparedStatement stmt = con.getPreparedStatement("registry.lookup");
+            synchronized (stmt) {
+                stmt.setLong(1, key.longHashCode());
+
+                try(ResultSet r = stmt.executeQuery()) {
+                    if(r.next()) {
+                        return r.getLong(1);
+                    }
+                }
+            }
+        } catch (SQLException e) {
+            log.error("error looking up key in temporary database table",e);
+        } finally {
+            releaseConnection(con);
+        }
+        return -1;
+    }
+
+
+    @Override
+    public void releaseTransaction(long transactionId) {
+        KiWiConnection con = aqcuireConnection();
+        try {
+            PreparedStatement stmt = con.getPreparedStatement("registry.release");
+            synchronized (stmt) {
+                stmt.setLong(1, transactionId);
+                stmt.executeUpdate();
+            }
+        } catch (SQLException e) {
+            log.error("error releasing key in temporary database table",e);
+        } finally {
+            releaseConnection(con);
+        }
+
+    }
+
+
+    @Override
+    public void deleteKey(IntArray key) {
+        KiWiConnection con = aqcuireConnection();
+        try {
+            PreparedStatement stmt = con.getPreparedStatement("registry.delete");
+            synchronized (stmt) {
+                stmt.setLong(1, key.longHashCode());
+                stmt.executeUpdate();
+            }
+        } catch (SQLException e) {
+            log.error("error deleting key in temporary database table",e);
+        } finally {
+            releaseConnection(con);
+        }
+
+    }
+
+
+    protected KiWiConnection aqcuireConnection() {
+        try {
+            return store.getPersistence().getConnection();
+        } catch(SQLException ex) {
+            log.error("could not acquire database connection", ex);
+            throw new RuntimeException(ex);
+        }
+    }
+
+    protected void releaseConnection(KiWiConnection con) {
+        try {
+            con.getJDBCConnection().commit();
+            con.close();
+        } catch (SQLException ex) {
+            log.error("could not release database connection", ex);
+            throw new RuntimeException(ex);
+        }
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/KiWiTripleRegistry.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/KiWiTripleRegistry.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/KiWiTripleRegistry.java
new file mode 100644
index 0000000..cd14fbb
--- /dev/null
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/KiWiTripleRegistry.java
@@ -0,0 +1,61 @@
+/*
+ * 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.marmotta.kiwi.persistence.registry;
+
+import org.apache.marmotta.commons.sesame.tripletable.IntArray;
+
+/**
+ * Add file description here!
+ *
+ * @author Sebastian Schaffert (sschaffert@apache.org)
+ */
+public interface KiWiTripleRegistry {
+
+    /**
+     * Register a key/triple id pair in the triple registry for the given transaction ID.
+     *
+     * @param key            the key identifying the triple arguments (subject, object, predicate, context)
+     * @param transactionId  the identifier of the transaction registering the triple id
+     * @param tripleId       the new triple identifier
+     */
+    void registerKey(IntArray key, long transactionId, long tripleId);
+
+    /**
+     * Check if another (or the same) transaction has already registered an ID for the triple with the
+     * given key. Returns -1 in case no other ID has been registered, or a value >0 otherwise.
+     *
+     * @param key            the key identifying the triple arguments (subject, object, predicate, context)
+     * @return               id of the triple or -1
+     */
+    long lookupKey(IntArray key);
+
+    /**
+     * Free all registry entries claimed by the transaction with the given identifier. Should remove or
+     * expire all registry entries to avoid unnecessary storage consumption.
+     *
+     * @param transactionId  the identifier of the transaction registering the triple id
+     */
+    void releaseTransaction(long transactionId);
+
+    /**
+     * Remove the key with the given key, e.g. when a statement is again deleted during a transaction.
+     *
+     * @param key            the key identifying the triple arguments (subject, object, predicate, context)
+     */
+    void deleteKey(IntArray key);
+}

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java
index 495e953..444af01 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java
@@ -17,17 +17,13 @@
  */
 package org.apache.marmotta.kiwi.sail;
 
-import net.sf.ehcache.Element;
-import net.sf.ehcache.constructs.blocking.CacheEntryFactory;
-import net.sf.ehcache.constructs.blocking.SelfPopulatingCache;
 import org.apache.marmotta.commons.sesame.model.LiteralCommons;
-import org.apache.marmotta.commons.sesame.model.LiteralKey;
 import org.apache.marmotta.commons.sesame.model.Namespaces;
 import org.apache.marmotta.commons.sesame.tripletable.IntArray;
 import org.apache.marmotta.commons.util.DateUtils;
 import org.apache.marmotta.kiwi.model.rdf.*;
 import org.apache.marmotta.kiwi.persistence.KiWiConnection;
-import org.apache.marmotta.kiwi.persistence.KiWiTripleRegistry;
+import org.apache.marmotta.kiwi.persistence.registry.DBTripleRegistry;
 import org.openrdf.model.*;
 import org.openrdf.model.impl.ContextStatementImpl;
 import org.slf4j.Logger;
@@ -54,43 +50,16 @@ public class KiWiValueFactory implements ValueFactory {
 
     private KiWiStore store;
 
-    private KiWiTripleRegistry registry;
+    private DBTripleRegistry registry;
 
     private String defaultContext;
 
-
-    protected SelfPopulatingCache literalCache;
-    protected SelfPopulatingCache uriCache;
-    protected SelfPopulatingCache bnodeCache;
-
     public KiWiValueFactory(KiWiStore store, String defaultContext) {
         anonIdGenerator = new Random();
-        registry        = new KiWiTripleRegistry(store);
+        registry        = new DBTripleRegistry(store);
 
         this.store          = store;
         this.defaultContext = defaultContext;
-
-        this.literalCache = new SelfPopulatingCache(store.getPersistence().getCacheManager().getLoaderCache(), new CacheEntryFactory() {
-            @Override
-            public Object createEntry(Object key) throws Exception {
-                return createLiteralInternal((LiteralKey) key);
-            }
-        });
-
-        this.uriCache = new SelfPopulatingCache(store.getPersistence().getCacheManager().getLoaderCache(), new CacheEntryFactory() {
-            @Override
-            public Object createEntry(Object key) throws Exception {
-                return createURIInternal(key.toString().intern());
-            }
-        });
-
-        this.bnodeCache = new SelfPopulatingCache(store.getPersistence().getCacheManager().getLoaderCache(), new CacheEntryFactory() {
-            @Override
-            public Object createEntry(Object key) throws Exception {
-                return createBNodeInternal(key.toString().intern());
-            }
-        });
-
     }
 
     protected KiWiConnection aqcuireConnection() {
@@ -130,16 +99,6 @@ public class KiWiValueFactory implements ValueFactory {
      */
     @Override
     public URI createURI(String uri) {
-        Element e = uriCache.get(uri);
-        if(e != null) {
-            return (KiWiUriResource) e.getObjectValue();
-        } else {
-            log.error("could not load URI resource");
-            throw new IllegalStateException("database error, could not load URI resource");
-        }
-    }
-
-    private KiWiUriResource createURIInternal(String uri) {
         KiWiConnection connection = aqcuireConnection();
         try {
             // first look in the registry for newly created resources if the resource has already been created and
@@ -193,17 +152,6 @@ public class KiWiValueFactory implements ValueFactory {
      */
     @Override
     public BNode createBNode(String nodeID) {
-        Element e = bnodeCache.get(nodeID);
-        if(e != null) {
-            return (KiWiAnonResource) e.getObjectValue();
-        } else {
-            log.error("could not load BNode resource");
-            throw new IllegalStateException("database error, could not load anonymous resource");
-        }
-    }
-
-    private KiWiAnonResource createBNodeInternal(String nodeID) {
-
         KiWiConnection connection = aqcuireConnection();
         try {
             // first look in the registry for newly created resources if the resource has already been created and
@@ -311,30 +259,6 @@ public class KiWiValueFactory implements ValueFactory {
      * @return
      */
     private <T> KiWiLiteral createLiteral(T value, String lang, String type) {
-
-        if (lang != null) {
-            // FIXME: MARMOTTA-39 (no rdf:langString)
-            // type = LiteralCommons.getRDFLangStringType();
-        } else if (type == null) {
-            // FIXME: MARMOTTA-39 (no default datatype before RDF-1.1)
-            // type = LiteralCommons.getXSDType(value.getClass());
-        }
-        LiteralKey lkey = new LiteralKey(value,type, lang != null ? lang.intern() : null);
-
-
-        Element e = literalCache.get(lkey);
-        if(e != null) {
-            return (KiWiLiteral) e.getObjectValue();
-        } else {
-            log.error("could not load Literal value");
-            throw new IllegalStateException("database error, could not load literal value");
-        }
-    }
-
-    private KiWiLiteral createLiteralInternal(LiteralKey key) {
-        Object value = key.getValue();
-        String lang  = key.getLang();
-        String type  = key.getType();
         Locale locale;
         if(lang != null) {
             try {
@@ -655,8 +579,10 @@ public class KiWiValueFactory implements ValueFactory {
      */
     protected void removeStatement(KiWiTriple triple) {
         if(triple.getId() >= 0) {
+            IntArray cacheKey = IntArray.createSPOCKey(triple.getSubject(), triple.getPredicate(), triple.getObject(), triple.getContext());
+
             synchronized (registry) {
-                registry.deleteKey(triple.getId());
+                registry.deleteKey(cacheKey);
             }
         }
         triple.setDeleted(true);

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/h2/statements.properties
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/h2/statements.properties b/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/h2/statements.properties
index 3bdf5dc..3e37358 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/h2/statements.properties
+++ b/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/h2/statements.properties
@@ -83,4 +83,4 @@ gc.list_node_ids     = SELECT id FROM nodes WHERE svalue = ? AND ntype = ? AND i
 registry.lookup      = SELECT tripleId FROM registry WHERE tripleKey = ?  LIMIT 1
 registry.register    = INSERT INTO registry (tripleKey, tripleId, txId) VALUES (?,?,?)
 registry.release     = DELETE FROM registry WHERE txId = ?
-registry.delete      = DELETE FROM registry WHERE tripleId = ?
+registry.delete      = DELETE FROM registry WHERE tripleKey = ?

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/mysql/statements.properties
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/mysql/statements.properties b/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/mysql/statements.properties
index 44a1d38..11fe0fa 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/mysql/statements.properties
+++ b/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/mysql/statements.properties
@@ -81,4 +81,4 @@ gc.list_node_ids     = SELECT id FROM nodes WHERE svalue = ? AND ntype = ? AND i
 registry.lookup      = SELECT tripleId FROM registry WHERE tripleKey = ? LIMIT 1
 registry.register    = INSERT INTO registry (tripleKey, tripleId, txId) VALUES (?,?,?)
 registry.release     = DELETE FROM registry WHERE txId = ?
-registry.delete      = DELETE FROM registry WHERE tripleId = ?
+registry.delete      = DELETE FROM registry WHERE tripleKey = ?

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/pgsql/statements.properties
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/pgsql/statements.properties b/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/pgsql/statements.properties
index 09ac0d2..0d1c1de 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/pgsql/statements.properties
+++ b/libraries/kiwi/kiwi-triplestore/src/main/resources/org/apache/marmotta/kiwi/persistence/pgsql/statements.properties
@@ -78,4 +78,4 @@ gc.list_node_ids     = SELECT id FROM nodes WHERE svalue = ? AND ntype = CAST(?
 registry.lookup      = SELECT tripleId FROM registry WHERE tripleKey = ? LIMIT 1
 registry.register    = INSERT INTO registry (tripleKey, tripleId, txId) VALUES (?,?,?)
 registry.release     = DELETE FROM registry WHERE txId = ?
-registry.delete      = DELETE FROM registry WHERE tripleId = ?
+registry.delete      = DELETE FROM registry WHERE tripleKey = ?

http://git-wip-us.apache.org/repos/asf/marmotta/blob/c0c795b9/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 98e30ca..0de87b6 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -962,6 +962,23 @@
                 <artifactId>ehcache-core</artifactId>
                 <version>2.6.6</version>
             </dependency>
+
+            <dependency>
+                <groupId>org.infinispan</groupId>
+                <artifactId>infinispan-core</artifactId>
+                <version>6.0.0.Final</version>
+            </dependency>
+            <dependency>
+                <groupId>org.infinispan</groupId>
+                <artifactId>infinispan-cdi</artifactId>
+                <version>6.0.0.Final</version>
+            </dependency>
+            <dependency>
+                <groupId>org.infinispan</groupId>
+                <artifactId>infinispan-tree</artifactId>
+                <version>6.0.0.Final</version>
+            </dependency>
+
             <dependency>
                 <groupId>com.h2database</groupId>
                 <artifactId>h2</artifactId>


[5/7] git commit: bug fixes for MARMOTTA-388

Posted by ss...@apache.org.
bug fixes for MARMOTTA-388


Project: http://git-wip-us.apache.org/repos/asf/marmotta/repo
Commit: http://git-wip-us.apache.org/repos/asf/marmotta/commit/4ba0d1bf
Tree: http://git-wip-us.apache.org/repos/asf/marmotta/tree/4ba0d1bf
Diff: http://git-wip-us.apache.org/repos/asf/marmotta/diff/4ba0d1bf

Branch: refs/heads/develop
Commit: 4ba0d1bf4cd34b571f9729217dd6cf16fcc1fa91
Parents: 61c14a1
Author: Sebastian Schaffert <ss...@apache.org>
Authored: Tue Dec 17 14:44:54 2013 +0100
Committer: Sebastian Schaffert <ss...@apache.org>
Committed: Tue Dec 17 14:44:54 2013 +0100

----------------------------------------------------------------------
 .../marmotta/kiwi/caching/KiWiCacheManager.java |   2 +
 .../src/main/resources/jgroups-kiwi.xml         |  17 +++
 .../kiwi/test/LDCachePersistenceTest.java       |   8 +-
 .../test/rdf/TestLinkedDataProvider.java        |   1 +
 platform/marmotta-core/pom.xml                  |   2 +
 .../core/services/cache/CachingServiceImpl.java |   2 +
 .../ehcache/EHCacheStatisticsModule.java        | 140 -------------------
 .../marmotta/platform/core/util/CDIContext.java |   5 +-
 8 files changed, 28 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/marmotta/blob/4ba0d1bf/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
index f22a6ad..6c25504 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
@@ -72,6 +72,7 @@ public class KiWiCacheManager {
                         .machineId("instance-" + config.getDatacenterId())
                         .addProperty("configurationFile", "jgroups-kiwi.xml")
                     .globalJmxStatistics()
+                        .jmxDomain("org.apache.marmotta.kiwi")
                     .build();
 
 
@@ -95,6 +96,7 @@ public class KiWiCacheManager {
         } else {
             globalConfiguration = new GlobalConfigurationBuilder()
                     .globalJmxStatistics()
+                        .jmxDomain("org.apache.marmotta.kiwi")
                     .build();
 
             defaultConfiguration = new ConfigurationBuilder()

http://git-wip-us.apache.org/repos/asf/marmotta/blob/4ba0d1bf/libraries/kiwi/kiwi-triplestore/src/main/resources/jgroups-kiwi.xml
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/resources/jgroups-kiwi.xml b/libraries/kiwi/kiwi-triplestore/src/main/resources/jgroups-kiwi.xml
index aa5ce8c..fb9b7d3 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/resources/jgroups-kiwi.xml
+++ b/libraries/kiwi/kiwi-triplestore/src/main/resources/jgroups-kiwi.xml
@@ -1,3 +1,20 @@
+<!--
+  ~ 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.
+  -->
+
 <config xmlns="urn:org:jgroups"
         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
         xsi:schemaLocation="urn:org:jgroups http://www.jgroups.org/schema/JGroups-3.4.xsd">

http://git-wip-us.apache.org/repos/asf/marmotta/blob/4ba0d1bf/libraries/ldcache/ldcache-backend-kiwi/src/test/java/org/apache/marmotta/ldcache/backend/kiwi/test/LDCachePersistenceTest.java
----------------------------------------------------------------------
diff --git a/libraries/ldcache/ldcache-backend-kiwi/src/test/java/org/apache/marmotta/ldcache/backend/kiwi/test/LDCachePersistenceTest.java b/libraries/ldcache/ldcache-backend-kiwi/src/test/java/org/apache/marmotta/ldcache/backend/kiwi/test/LDCachePersistenceTest.java
index d066405..35f11b4 100644
--- a/libraries/ldcache/ldcache-backend-kiwi/src/test/java/org/apache/marmotta/ldcache/backend/kiwi/test/LDCachePersistenceTest.java
+++ b/libraries/ldcache/ldcache-backend-kiwi/src/test/java/org/apache/marmotta/ldcache/backend/kiwi/test/LDCachePersistenceTest.java
@@ -28,11 +28,7 @@ import org.apache.marmotta.kiwi.persistence.pgsql.PostgreSQLDialect;
 import org.apache.marmotta.ldcache.backend.kiwi.model.KiWiCacheEntry;
 import org.apache.marmotta.ldcache.backend.kiwi.persistence.LDCachingKiWiPersistence;
 import org.apache.marmotta.ldcache.backend.kiwi.persistence.LDCachingKiWiPersistenceConnection;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
+import org.junit.*;
 import org.junit.rules.TestWatcher;
 import org.junit.runner.Description;
 import org.junit.runner.RunWith;
@@ -171,7 +167,7 @@ public class LDCachePersistenceTest {
         LDCachingKiWiPersistenceConnection connection = vpersistence.getConnection();
         try {
             Assert.assertThat(connection.getDatabaseTables(), hasItems("ldcache_entries"));
-            Assert.assertEquals(2, connection.getDatabaseVersion());
+            Assert.assertEquals(3, connection.getDatabaseVersion());
 
             connection.commit();
         } finally {

http://git-wip-us.apache.org/repos/asf/marmotta/blob/4ba0d1bf/libraries/ldclient/ldclient-provider-rdf/src/test/java/org/apache/marmotta/ldclient/test/rdf/TestLinkedDataProvider.java
----------------------------------------------------------------------
diff --git a/libraries/ldclient/ldclient-provider-rdf/src/test/java/org/apache/marmotta/ldclient/test/rdf/TestLinkedDataProvider.java b/libraries/ldclient/ldclient-provider-rdf/src/test/java/org/apache/marmotta/ldclient/test/rdf/TestLinkedDataProvider.java
index f6fa7cf..bc9277b 100644
--- a/libraries/ldclient/ldclient-provider-rdf/src/test/java/org/apache/marmotta/ldclient/test/rdf/TestLinkedDataProvider.java
+++ b/libraries/ldclient/ldclient-provider-rdf/src/test/java/org/apache/marmotta/ldclient/test/rdf/TestLinkedDataProvider.java
@@ -117,6 +117,7 @@ public class TestLinkedDataProvider {
      *
      */
     @Test
+    @Ignore("test failing for the moment because the data returned by the service is wrong")
     public void testRDFOhloh() throws Exception {
     	Assume.assumeTrue(ldclient.ping(MARMOTTA));
         ClientResponse response = ldclient.retrieveResource(MARMOTTA);

http://git-wip-us.apache.org/repos/asf/marmotta/blob/4ba0d1bf/platform/marmotta-core/pom.xml
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/pom.xml b/platform/marmotta-core/pom.xml
index d38d15e..3cad061 100644
--- a/platform/marmotta-core/pom.xml
+++ b/platform/marmotta-core/pom.xml
@@ -341,10 +341,12 @@
             <groupId>org.infinispan</groupId>
             <artifactId>infinispan-core</artifactId>
         </dependency>
+        <!-- TODO: we still use our own CDI implementation here...
         <dependency>
             <groupId>org.infinispan</groupId>
             <artifactId>infinispan-cdi</artifactId>
         </dependency>
+        -->
         <dependency>
             <groupId>javax.validation</groupId>
             <artifactId>validation-api</artifactId>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/4ba0d1bf/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
index 729a563..459d63c 100644
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
+++ b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
@@ -86,6 +86,7 @@ public class CachingServiceImpl implements CachingService {
                         .machineId(configurationService.getServerName())
                         .addProperty("configurationFile", "jgroups-marmotta.xml")
                     .globalJmxStatistics()
+                        .jmxDomain("org.apache.marmotta.platform")
                     .build();
 
 
@@ -109,6 +110,7 @@ public class CachingServiceImpl implements CachingService {
         } else {
             globalConfiguration = new GlobalConfigurationBuilder()
                     .globalJmxStatistics()
+                        .jmxDomain("org.apache.marmotta.platform")
                     .build();
 
             defaultConfiguration = new ConfigurationBuilder()

http://git-wip-us.apache.org/repos/asf/marmotta/blob/4ba0d1bf/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/statistics/ehcache/EHCacheStatisticsModule.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/statistics/ehcache/EHCacheStatisticsModule.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/statistics/ehcache/EHCacheStatisticsModule.java
deleted file mode 100644
index a4f4f8e..0000000
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/statistics/ehcache/EHCacheStatisticsModule.java
+++ /dev/null
@@ -1,140 +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.marmotta.platform.core.services.statistics.ehcache;
-
-import org.apache.marmotta.platform.core.api.cache.CachingService;
-import org.apache.marmotta.platform.core.api.statistics.StatisticsModule;
-import net.sf.ehcache.Ehcache;
-import net.sf.ehcache.Statistics;
-import org.slf4j.Logger;
-
-import javax.annotation.PostConstruct;
-import javax.inject.Inject;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Offer statistical information about the EHCache caches used in the KiWi System
- * <p/>
- * User: sschaffe
- */
-public class EHCacheStatisticsModule implements StatisticsModule {
-
-    @Inject
-    private Logger log;
-
-    @Inject
-    private CachingService cachingService;
-
-    @PostConstruct
-    public void initialize() {
-
-
-    }
-
-     /**
-     * Enable this module. Depending on the type of information, this may involve additional runtime overhead.
-     */
-    @Override
-    public void enable() {
-        for(String cacheName : cachingService.getCacheNames()) {
-             Ehcache cache = cachingService.getCacheByName(cacheName);
-             if(cache != null) {
-                 cache.setStatisticsEnabled(true);
-                 cache.setStatisticsAccuracy(Statistics.STATISTICS_ACCURACY_GUARANTEED);
-             }
-         }
-    }
-
-    /**
-     * Disable this module.
-     */
-    @Override
-    public void disable() {
-        for(String cacheName : cachingService.getCacheNames()) {
-             Ehcache cache = cachingService.getCacheByName(cacheName);
-             if(cache != null) {
-                 cache.setStatisticsEnabled(false);
-                 cache.setStatisticsAccuracy(Statistics.STATISTICS_ACCURACY_NONE);
-             }
-         }
-    }
-
-    /**
-     * Return true if the module is enabled.
-     *
-     * @return
-     */
-    @Override
-    public boolean isEnabled() {
-        return true;
-    }
-
-    /**
-     * Return all names of properties supported by this module.
-     *
-     * @return
-     */
-    @Override
-    public List<String> getPropertyNames() {
-
-        List<String> propertyNames = new LinkedList<String>();
-        for(String cacheName : cachingService.getCacheNames()) {
-            propertyNames.add(cacheName + " hits");
-            propertyNames.add(cacheName + " misses");
-            propertyNames.add(cacheName + " size");
-        }
-        return propertyNames;
-    }
-
-    /**
-     * Return the statistics as a map from key to value
-     *
-     * @return
-     */
-    @Override
-    public Map<String, String> getStatistics() {
-
-        LinkedHashMap<String,String> result = new LinkedHashMap<String, String>();
-        for(String cacheName : cachingService.getCacheNames()) {
-            Ehcache cache = cachingService.getCacheByName(cacheName);
-            if(cache != null) {
-                Statistics stat = cache.getStatistics();
-
-                result.put(cacheName + " hits",""+stat.getCacheHits());
-                result.put(cacheName + " misses",""+stat.getCacheMisses());
-                result.put(cacheName + " size",""+stat.getObjectCount());
-            } else {
-                log.warn("cache with name {} does not exist",cacheName);
-            }
-        }
-        return result;
-    }
-
-    /**
-     * Return the display name of the statistics module.
-     *
-     * @return
-     */
-    @Override
-    public String getName() {
-        return "Cache Statistics";
-    }
-}

http://git-wip-us.apache.org/repos/asf/marmotta/blob/4ba0d1bf/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/util/CDIContext.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/util/CDIContext.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/util/CDIContext.java
index 76a5ed4..c35e570 100644
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/util/CDIContext.java
+++ b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/util/CDIContext.java
@@ -65,9 +65,8 @@ public class CDIContext {
             try {
                 return (BeanManager) new InitialContext().lookup(location);
             } catch (NameNotFoundException e) {
-                log.error(
-                        "NameNotFoundException for path {}",
-                        location, e);
+                // do nothing: it is ok to throw an exception here because we will look anyways in other locations and
+                // throw an exception at the end of the method
             } catch (NamingException e) {
                 log.error(
                         "naming exception for path {}; this probably means that JNDI is not set up properly (see e.g. http://code.google.com/p/lmf/wiki/InstallationSetup#Specific_Settings_for_Tomcat )",


[4/7] git commit: finished MARMOTTA-388: transition from EHCache to Infinispan

Posted by ss...@apache.org.
finished MARMOTTA-388: transition from EHCache to Infinispan


Project: http://git-wip-us.apache.org/repos/asf/marmotta/repo
Commit: http://git-wip-us.apache.org/repos/asf/marmotta/commit/61c14a19
Tree: http://git-wip-us.apache.org/repos/asf/marmotta/tree/61c14a19
Diff: http://git-wip-us.apache.org/repos/asf/marmotta/diff/61c14a19

Branch: refs/heads/develop
Commit: 61c14a19f2b38b65743e8ef3991b13f7b4ec3cac
Parents: af9a800
Author: Sebastian Schaffert <ss...@apache.org>
Authored: Tue Dec 17 14:13:17 2013 +0100
Committer: Sebastian Schaffert <ss...@apache.org>
Committed: Tue Dec 17 14:13:17 2013 +0100

----------------------------------------------------------------------
 .../marmotta/kiwi/caching/KiWiCacheManager.java |  27 +++-
 .../src/main/resources/jgroups-kiwi.xml         |  74 +++++++++++
 .../backend/kiwi/model/KiWiCacheEntry.java      |   4 +-
 .../LDCachingKiWiPersistenceConnection.java     |  27 ++--
 parent/pom.xml                                  |   5 -
 .../backend/kiwi/KiWiStoreProvider.java         |   2 +-
 .../main/resources/config-defaults.properties   |   3 +-
 .../resources/config-descriptions.properties    |   3 -
 .../platform/core/api/cache/CachingService.java |  15 ++-
 .../core/api/modules/ResourceEntry.java         |   3 +-
 .../core/logging/CacheLoggingModule.java        |   2 +-
 .../core/services/cache/CachingServiceImpl.java | 133 ++++++++++++++-----
 .../services/http/HttpClientServiceImpl.java    | 111 +++++++++++++++-
 .../modules/MarmottaResourceServiceImpl.java    |  12 +-
 .../main/resources/config-defaults.properties   |  10 ++
 .../resources/config-descriptions.properties    |   7 +
 .../src/main/resources/jgroups-marmotta.xml     |  74 +++++++++++
 .../user/services/AccountServiceImpl.java       |  21 ++-
 18 files changed, 438 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
index 8af3ba6..f22a6ad 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
@@ -70,7 +70,7 @@ public class KiWiCacheManager {
                         .defaultTransport()
                         .clusterName(config.getName())
                         .machineId("instance-" + config.getDatacenterId())
-                        .addProperty("configurationFile", "jgroups-udp.xml")
+                        .addProperty("configurationFile", "jgroups-kiwi.xml")
                     .globalJmxStatistics()
                     .build();
 
@@ -154,6 +154,28 @@ public class KiWiCacheManager {
 
 
     /**
+     * Return the triple id -> triple cache from the cache manager. This cache is used for speeding up the
+     * construction of query results.
+     *
+     * @return
+     */
+    public Cache getQueryCache() {
+        if(!cacheManager.cacheExists(TRIPLE_CACHE)) {
+            Configuration tripleConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .eviction()
+                        .maxEntries(100000)
+                    .expiration()
+                        .lifespan(5, TimeUnit.MINUTES)
+                        .maxIdle(60, TimeUnit.SECONDS)
+                    .build();
+            cacheManager.defineConfiguration(TRIPLE_CACHE, tripleConfiguration);
+        }
+        return cacheManager.getCache(TRIPLE_CACHE);
+    }
+
+
+
+    /**
      * Return the uri -> KiWiUriResource cache from the cache manager. This cache is used when constructing new
      * KiWiUriResources to avoid a database lookup.
      *
@@ -318,7 +340,8 @@ public class KiWiCacheManager {
             String cacheName = iterator.next();
             Cache<String,Object> cache = cacheManager.getCache(cacheName);
             cache.clear();
-        }      }
+        }
+    }
 
     /**
      * Shutdown this cache manager instance. Will shutdown the underlying EHCache cache manager.

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/libraries/kiwi/kiwi-triplestore/src/main/resources/jgroups-kiwi.xml
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/resources/jgroups-kiwi.xml b/libraries/kiwi/kiwi-triplestore/src/main/resources/jgroups-kiwi.xml
new file mode 100644
index 0000000..aa5ce8c
--- /dev/null
+++ b/libraries/kiwi/kiwi-triplestore/src/main/resources/jgroups-kiwi.xml
@@ -0,0 +1,74 @@
+<config xmlns="urn:org:jgroups"
+        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+        xsi:schemaLocation="urn:org:jgroups http://www.jgroups.org/schema/JGroups-3.4.xsd">
+   <UDP
+         mcast_addr="${jgroups.udp.mcast_addr:228.6.7.8}"
+         mcast_port="${jgroups.udp.mcast_port:46655}"
+         tos="8"
+         ucast_recv_buf_size="20m"
+         ucast_send_buf_size="640k"
+         mcast_recv_buf_size="25m"
+         mcast_send_buf_size="640k"
+         loopback="true"
+         max_bundle_size="31k"
+         ip_ttl="${jgroups.udp.ip_ttl:2}"
+         enable_diagnostics="false"
+         bundler_type="old"
+
+         thread_naming_pattern="pl"
+
+         thread_pool.enabled="true"
+         thread_pool.min_threads="2"
+         thread_pool.max_threads="30"
+         thread_pool.keep_alive_time="60000"
+         thread_pool.queue_enabled="true"
+         thread_pool.queue_max_size="100"
+         thread_pool.rejection_policy="Discard"
+
+         oob_thread_pool.enabled="true"
+         oob_thread_pool.min_threads="2"
+         oob_thread_pool.max_threads="30"
+         oob_thread_pool.keep_alive_time="60000"
+         oob_thread_pool.queue_enabled="false"
+         oob_thread_pool.queue_max_size="100"
+         oob_thread_pool.rejection_policy="Discard"
+
+         internal_thread_pool.enabled="true"
+         internal_thread_pool.min_threads="1"
+         internal_thread_pool.max_threads="10"
+         internal_thread_pool.keep_alive_time="60000"
+         internal_thread_pool.queue_enabled="true"
+         internal_thread_pool.queue_max_size="100"
+         internal_thread_pool.rejection_policy="Discard"
+         />
+
+   <PING timeout="3000" num_initial_members="3"/>
+   <MERGE2 max_interval="30000" min_interval="10000"/>
+
+   <FD_SOCK/>
+   <FD_ALL timeout="15000" interval="3000"/>
+   <VERIFY_SUSPECT timeout="1500"/>
+
+   <pbcast.NAKACK2
+                    xmit_interval="1000"
+                    xmit_table_num_rows="100"
+                    xmit_table_msgs_per_row="10000"
+                    xmit_table_max_compaction_time="10000"
+                    max_msg_batch_size="100"/>
+   <UNICAST3
+              xmit_interval="500"
+              xmit_table_num_rows="20"
+              xmit_table_msgs_per_row="10000"
+              xmit_table_max_compaction_time="10000"
+              max_msg_batch_size="100"
+              conn_expiry_timeout="0"/>
+
+   <pbcast.STABLE stability_delay="500" desired_avg_gossip="5000" max_bytes="1m"/>
+   <pbcast.GMS print_local_addr="false" join_timeout="3000" view_bundling="true"/>
+   <tom.TOA/> <!-- the TOA is only needed for total order transactions-->
+
+   <UFC max_credits="2m" min_threshold="0.40"/>
+   <MFC max_credits="2m" min_threshold="0.40"/>
+   <FRAG2 frag_size="30k"  />
+   <RSVP timeout="60000" resend_interval="500" ack_on_delivery="false" />
+</config>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/model/KiWiCacheEntry.java
----------------------------------------------------------------------
diff --git a/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/model/KiWiCacheEntry.java b/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/model/KiWiCacheEntry.java
index 265970d..fb57cbd 100644
--- a/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/model/KiWiCacheEntry.java
+++ b/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/model/KiWiCacheEntry.java
@@ -19,12 +19,14 @@ package org.apache.marmotta.ldcache.backend.kiwi.model;
 
 import org.apache.marmotta.ldcache.model.CacheEntry;
 
+import java.io.Serializable;
+
 /**
  * Add file description here!
  * <p/>
  * Author: Sebastian Schaffert (sschaffert@apache.org)
  */
-public class KiWiCacheEntry extends CacheEntry {
+public class KiWiCacheEntry extends CacheEntry implements Serializable {
 
     Long id;
 

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/persistence/LDCachingKiWiPersistenceConnection.java
----------------------------------------------------------------------
diff --git a/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/persistence/LDCachingKiWiPersistenceConnection.java b/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/persistence/LDCachingKiWiPersistenceConnection.java
index bd79539..a0691d2 100644
--- a/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/persistence/LDCachingKiWiPersistenceConnection.java
+++ b/libraries/ldcache/ldcache-backend-kiwi/src/main/java/org/apache/marmotta/ldcache/backend/kiwi/persistence/LDCachingKiWiPersistenceConnection.java
@@ -18,8 +18,6 @@
 package org.apache.marmotta.ldcache.backend.kiwi.persistence;
 
 import info.aduna.iteration.CloseableIteration;
-import net.sf.ehcache.Cache;
-import net.sf.ehcache.Element;
 import org.apache.marmotta.kiwi.model.rdf.KiWiNode;
 import org.apache.marmotta.kiwi.model.rdf.KiWiResource;
 import org.apache.marmotta.kiwi.persistence.KiWiConnection;
@@ -27,6 +25,7 @@ import org.apache.marmotta.kiwi.persistence.util.ResultSetIteration;
 import org.apache.marmotta.kiwi.persistence.util.ResultTransformerFunction;
 import org.apache.marmotta.ldcache.backend.kiwi.model.KiWiCacheEntry;
 import org.apache.marmotta.ldcache.model.CacheEntry;
+import org.infinispan.Cache;
 import org.openrdf.model.URI;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,13 +52,13 @@ public class LDCachingKiWiPersistenceConnection  {
     /**
      * Cache entries by resource
      */
-    private Cache entryResourceCache;
+    private Cache<String,KiWiCacheEntry> entryResourceCache;
 
 
     /**
      * Cache entries by ID
      */
-    private Cache entryIdCache;
+    private Cache<Long,KiWiCacheEntry> entryIdCache;
 
 
     public LDCachingKiWiPersistenceConnection(KiWiConnection connection) throws SQLException {
@@ -72,11 +71,11 @@ public class LDCachingKiWiPersistenceConnection  {
     public KiWiCacheEntry constructCacheEntry(ResultSet row) throws SQLException {
         Long id = row.getLong("id");
 
-        Element cached = entryIdCache.get(id);
+        KiWiCacheEntry cached = entryIdCache.get(id);
 
         // lookup element in cache first, so we can avoid reconstructing it if it is already there
         if(cached != null) {
-            return (KiWiCacheEntry)cached.getObjectValue();
+            return cached;
         }
 
         KiWiCacheEntry entry = new KiWiCacheEntry();
@@ -87,8 +86,8 @@ public class LDCachingKiWiPersistenceConnection  {
         entry.setResource((URI) connection.loadNodeById(row.getLong("resource_id")));
         entry.setTripleCount(row.getInt("triple_count"));
 
-        entryIdCache.put(new Element(id,entry));
-        entryResourceCache.put(new Element(entry.getResource().stringValue(),entry));
+        entryIdCache.put(id,entry);
+        entryResourceCache.put(entry.getResource().stringValue(),entry);
 
         return entry;
     }
@@ -103,11 +102,11 @@ public class LDCachingKiWiPersistenceConnection  {
      */
     public KiWiCacheEntry getCacheEntry(String uri) throws SQLException {
 
-        Element cached = entryResourceCache.get(uri);
+        KiWiCacheEntry cached = entryResourceCache.get(uri);
 
         // lookup element in cache first, so we can avoid reconstructing it if it is already there
         if(cached != null) {
-            return (KiWiCacheEntry)cached.getObjectValue();
+            return cached;
         }
 
         PreparedStatement query = connection.getPreparedStatement("load.entry_by_uri");
@@ -169,8 +168,8 @@ public class LDCachingKiWiPersistenceConnection  {
 
         log.debug("persisted ld-cache entry with id {}", kEntry.getId());
         
-        entryIdCache.put(new Element(kEntry.getId(),kEntry));
-        entryResourceCache.put(new Element(kEntry.getResource().stringValue(),kEntry));
+        entryIdCache.put(kEntry.getId(),kEntry);
+        entryResourceCache.put(kEntry.getResource().stringValue(),kEntry);
 
     }
 
@@ -205,11 +204,11 @@ public class LDCachingKiWiPersistenceConnection  {
         deleteEntry.setString(1,uri);
         deleteEntry.executeUpdate();
 
-        Element cached = entryResourceCache.get(uri);
+        KiWiCacheEntry cached = entryResourceCache.get(uri);
 
         if(cached != null) {
             entryResourceCache.remove(uri);
-            entryIdCache.remove(((KiWiCacheEntry) cached.getObjectValue()).getId());
+            entryIdCache.remove(cached.getId());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 62cf99f..8be8f54 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -1449,11 +1449,6 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.marmotta</groupId>
-                <artifactId>ldcache-backend-ehcache</artifactId>
-                <version>${project.version}</version>
-            </dependency>
-            <dependency>
-                <groupId>org.apache.marmotta</groupId>
                 <artifactId>ldcache-sail-kiwi</artifactId>
                 <version>${project.version}</version>
             </dependency>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
----------------------------------------------------------------------
diff --git a/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java b/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
index 979c806..b7d63e2 100644
--- a/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
+++ b/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
@@ -112,7 +112,7 @@ public class KiWiStoreProvider implements StoreProvider {
         configuration.setDatacenterId(configurationService.getIntConfiguration(DATACENTER_ID,0));
         configuration.setFulltextEnabled(configurationService.getBooleanConfiguration(FULLTEXT_ENABLED, true));
         configuration.setFulltextLanguages(configurationService.getListConfiguration(FULLTEXT_LANGUAGES, ImmutableList.of("en")));
-        configuration.setClustered(configurationService.getBooleanConfiguration("database.clustered", false));
+        configuration.setClustered(configurationService.getBooleanConfiguration("clustering.enabled", false));
 
         if("native".equalsIgnoreCase(configurationService.getStringConfiguration(SPARQL_STRATEGY))) {
             return new KiWiSparqlSail(new KiWiStore(configuration));

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties
----------------------------------------------------------------------
diff --git a/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties b/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties
index 65c013d..6ee6eb6 100644
--- a/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties
+++ b/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties
@@ -69,6 +69,5 @@ database.mysql.driver = com.mysql.jdbc.Driver
 database.mysql.url = jdbc:mysql://localhost:3306/lmf?useUnicode=true&characterEncoding=UTF-8&rewriteBatchedStatements=true
 
 
-# Turn on cluster-specific configuration options (e.g. replicated and distributed caching, synchronization, ...)
-database.clustered = false
+
 

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties
----------------------------------------------------------------------
diff --git a/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties b/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties
index c214f3f..49c53a2 100644
--- a/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties
+++ b/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties
@@ -53,6 +53,3 @@ database.fulltext.enabled.type = java.lang.Boolean
 
 database.fulltext.languages.description = list of languages supported by fulltext search; a fulltext index will be created for each language (PostgreSQL only)
 database.fulltext.languages.type = java.util.List
-
-database.clustered.description = Turn on cluster-specific configuration options (e.g. replicated and distributed caching, synchronization, ...)
-database.clustered.type = java.lang.Boolean

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/cache/CachingService.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/cache/CachingService.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/cache/CachingService.java
index 4329a3e..00cb18f 100644
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/cache/CachingService.java
+++ b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/cache/CachingService.java
@@ -17,10 +17,11 @@
  */
 package org.apache.marmotta.platform.core.api.cache;
 
-import net.sf.ehcache.CacheManager;
-import net.sf.ehcache.Ehcache;
+import org.infinispan.Cache;
+import org.infinispan.manager.EmbeddedCacheManager;
 
 import javax.enterprise.inject.spi.InjectionPoint;
+import java.util.Set;
 
 /**
  * Add file description here!
@@ -29,13 +30,15 @@ import javax.enterprise.inject.spi.InjectionPoint;
  */
 public interface CachingService {
 
-    public Ehcache getCache(InjectionPoint injectionPoint);
 
-    public String[] getCacheNames();
+
+    public Cache getCache(InjectionPoint injectionPoint);
+
+    public Set<String> getCacheNames();
 
     public void clearAll();
 
-    Ehcache getCacheByName(String cacheName);
+    public Cache getCacheByName(String cacheName);
 
-    CacheManager getCacheManager();
+    public EmbeddedCacheManager getCacheManager();
 }

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/modules/ResourceEntry.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/modules/ResourceEntry.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/modules/ResourceEntry.java
index 192e115..d3d4b39 100644
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/modules/ResourceEntry.java
+++ b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/api/modules/ResourceEntry.java
@@ -17,6 +17,7 @@
  */
 package org.apache.marmotta.platform.core.api.modules;
 
+import java.io.Serializable;
 import java.net.URL;
 
 /**
@@ -24,7 +25,7 @@ import java.net.URL;
  * <p/>
  * User: sschaffe
  */
-public class ResourceEntry {
+public class ResourceEntry implements Serializable {
 
     /**
      * The path relative to the web application root under which this resource is accessed.

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/logging/CacheLoggingModule.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/logging/CacheLoggingModule.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/logging/CacheLoggingModule.java
index 38eb5f7..af6709c 100644
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/logging/CacheLoggingModule.java
+++ b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/logging/CacheLoggingModule.java
@@ -58,6 +58,6 @@ public class CacheLoggingModule extends BaseLoggingModule {
      */
     @Override
     public Collection<String> getPackages() {
-        return ImmutableSet.of("net.sf.ehcache", "org.apache.marmotta.platform.core.services.cache");
+        return ImmutableSet.of("org.infinispan", "org.apache.marmotta.platform.core.services.cache");
     }
 }

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
index f1e9d80..729a563 100644
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
+++ b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
@@ -18,10 +18,19 @@
 package org.apache.marmotta.platform.core.services.cache;
 
 import org.apache.marmotta.platform.core.api.cache.CachingService;
+import org.apache.marmotta.platform.core.api.config.ConfigurationService;
 import org.apache.marmotta.platform.core.events.SystemRestartingEvent;
 import org.apache.marmotta.platform.core.qualifiers.cache.MarmottaCache;
-import net.sf.ehcache.CacheManager;
-import net.sf.ehcache.Ehcache;
+import org.infinispan.Cache;
+import org.infinispan.configuration.cache.CacheMode;
+import org.infinispan.configuration.cache.Configuration;
+import org.infinispan.configuration.cache.ConfigurationBuilder;
+import org.infinispan.configuration.global.GlobalConfiguration;
+import org.infinispan.configuration.global.GlobalConfigurationBuilder;
+import org.infinispan.distribution.ch.SyncConsistentHashFactory;
+import org.infinispan.eviction.EvictionStrategy;
+import org.infinispan.manager.DefaultCacheManager;
+import org.infinispan.manager.EmbeddedCacheManager;
 import org.slf4j.Logger;
 
 import javax.annotation.PostConstruct;
@@ -31,7 +40,9 @@ import javax.enterprise.event.Observes;
 import javax.enterprise.inject.Produces;
 import javax.enterprise.inject.spi.InjectionPoint;
 import javax.inject.Inject;
-import java.net.URL;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 /**
  * A service that offers a EHCache system cache implementation for use by other components
@@ -47,8 +58,16 @@ public class CachingServiceImpl implements CachingService {
     @Inject
     private Logger log;
 
+    @Inject
+    private ConfigurationService configurationService;
+
+    private EmbeddedCacheManager cacheManager;
+
+    private GlobalConfiguration globalConfiguration;
+
+    // default configuration: distributed cache, 100000 entries, 300 seconds expiration, 60 seconds idle
+    private Configuration defaultConfiguration;
 
-    private CacheManager manager;
 
     public CachingServiceImpl() {
     }
@@ -56,15 +75,57 @@ public class CachingServiceImpl implements CachingService {
 
     @PostConstruct
     public void initialize() {
-        URL url = this.getClass().getClassLoader().getResource("ehcache-marmotta.xml");
+        boolean clustered = configurationService.getBooleanConfiguration("clustering.enabled", false);
+
+        log.info("Apache Marmotta Caching Service starting up ({}) ...", clustered ? "clustering" : "single host" );
+        if(clustered) {
+            globalConfiguration = new GlobalConfigurationBuilder()
+                    .transport()
+                        .defaultTransport()
+                        .clusterName(configurationService.getStringConfiguration("clustering.name", "Marmotta"))
+                        .machineId(configurationService.getServerName())
+                        .addProperty("configurationFile", "jgroups-marmotta.xml")
+                    .globalJmxStatistics()
+                    .build();
+
+
+            defaultConfiguration = new ConfigurationBuilder()
+                    .clustering()
+                        .cacheMode(CacheMode.DIST_ASYNC)
+                        .async()
+                        .l1()
+                            .lifespan(25, TimeUnit.SECONDS)
+                        .hash()
+                            .numOwners(2)
+                            .numSegments(100)
+                            .consistentHashFactory(new SyncConsistentHashFactory())
+                    .eviction()
+                        .strategy(EvictionStrategy.LIRS)
+                        .maxEntries(1000)
+                    .expiration()
+                        .lifespan(5, TimeUnit.MINUTES)
+                        .maxIdle(1, TimeUnit.MINUTES)
+                    .build();
+        } else {
+            globalConfiguration = new GlobalConfigurationBuilder()
+                    .globalJmxStatistics()
+                    .build();
+
+            defaultConfiguration = new ConfigurationBuilder()
+                    .clustering()
+                        .cacheMode(CacheMode.LOCAL)
+                    .eviction()
+                        .strategy(EvictionStrategy.LIRS)
+                        .maxEntries(1000)
+                    .expiration()
+                        .lifespan(5, TimeUnit.MINUTES)
+                        .maxIdle(1, TimeUnit.MINUTES)
+                    .build();
 
-        // backwards compatibility
-        if(url == null) {
-            url = this.getClass().getClassLoader().getResource("ehcache-lmf.xml");
         }
 
-        log.info("Apache Marmotta Caching Service starting up (configuration at {}) ...",url);
-        manager = CacheManager.create(url);
+
+        cacheManager = new DefaultCacheManager(globalConfiguration, defaultConfiguration, true);
     }
 
     /**
@@ -82,35 +143,39 @@ public class CachingServiceImpl implements CachingService {
      */
     @Override
     @Produces @MarmottaCache("")
-    public Ehcache getCache(InjectionPoint injectionPoint) {
+    public Cache getCache(InjectionPoint injectionPoint) {
         String cacheName = injectionPoint.getAnnotated().getAnnotation(MarmottaCache.class).value();
 
         return getCacheByName(cacheName);
     }
 
 
-    @Override
-    public Ehcache getCacheByName(String cacheName) {
-        if(!manager.cacheExists(cacheName)) {
-            log.info("added new cache with name {}",cacheName);
-            manager.addCache(cacheName);
-        }
+    /**
+     * Allow CDI injection of the default cache
+     * @return
+     */
+    @Produces
+    public Configuration getDefaultConfiguration() {
+        return defaultConfiguration;
+    }
 
-        Ehcache cache = manager.getEhcache(cacheName);
-        cache.setStatisticsEnabled(true);
 
-        return cache;
+    @Override
+    public Cache getCacheByName(String cacheName) {
+        return cacheManager.getCache(cacheName, true);
     }
 
 
     @Override
-    public String[] getCacheNames() {
-        return manager.getCacheNames();
+    public Set<String> getCacheNames() {
+        return cacheManager.getCacheNames();
     }
 
     @Override
-    public CacheManager getCacheManager() {
-        return manager;
+    @Produces
+    @ApplicationScoped
+    public EmbeddedCacheManager getCacheManager() {
+        return cacheManager;
     }
 
 
@@ -120,27 +185,27 @@ public class CachingServiceImpl implements CachingService {
      */
     public void systemRestart(@Observes SystemRestartingEvent e) {
         log.warn("system restarted, flushing caches ...");
-        manager.clearAll();
+        cacheManager.stop();
+        cacheManager.start();
     }
 
 
     @Override
     public void clearAll() {
-        manager.clearAll();
+        Set<String> set =  cacheManager.getCacheNames();
+        Iterator<String> iterator =  set.iterator();
+        while(iterator.hasNext()){
+            String cacheName = iterator.next();
+            Cache<String,Object> cache = cacheManager.getCache(cacheName);
+            cache.clear();
+        }
     }
 
 
     @PreDestroy
     public void destroy() {
         log.info("Apache Marmotta Caching Service shutting down ...");
-        /*
-        for(String cacheName : manager.getCacheNames()) {
-            log.info("Disposing cache {} ...",cacheName);
-            Cache cache = manager.getCache(cacheName);
-            cache.dispose();
-        }
-         */
-        manager.shutdown();
+        cacheManager.stop();
         log.info("Apache Marmotta Caching Service shut down successfully.");
     }
 }

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/http/HttpClientServiceImpl.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/http/HttpClientServiceImpl.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/http/HttpClientServiceImpl.java
index afda0df..7d27576 100644
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/http/HttpClientServiceImpl.java
+++ b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/http/HttpClientServiceImpl.java
@@ -17,14 +17,12 @@
  */
 package org.apache.marmotta.platform.core.services.http;
 
-import net.sf.ehcache.Ehcache;
 import org.apache.http.*;
 import org.apache.http.client.ClientProtocolException;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.HttpRequestRetryHandler;
 import org.apache.http.client.ResponseHandler;
-import org.apache.http.client.cache.CacheResponseStatus;
-import org.apache.http.client.cache.HttpCacheStorage;
+import org.apache.http.client.cache.*;
 import org.apache.http.client.methods.*;
 import org.apache.http.client.params.ClientPNames;
 import org.apache.http.conn.ClientConnectionManager;
@@ -38,7 +36,7 @@ import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.http.impl.client.DefaultRedirectStrategy;
 import org.apache.http.impl.client.cache.CacheConfig;
 import org.apache.http.impl.client.cache.CachingHttpClient;
-import org.apache.http.impl.client.cache.ehcache.EhcacheHttpCacheStorage;
+import org.apache.http.impl.client.cache.DefaultHttpCacheEntrySerializer;
 import org.apache.http.impl.conn.PoolingClientConnectionManager;
 import org.apache.http.params.*;
 import org.apache.http.pool.PoolStats;
@@ -56,14 +54,16 @@ import org.apache.marmotta.platform.core.qualifiers.cache.MarmottaCache;
 import org.apache.marmotta.platform.core.services.http.response.LastModifiedResponseHandler;
 import org.apache.marmotta.platform.core.services.http.response.StatusCodeResponseHandler;
 import org.apache.marmotta.platform.core.services.http.response.StringBodyResponseHandler;
+import org.infinispan.Cache;
 import org.slf4j.Logger;
 
 import javax.annotation.PostConstruct;
 import javax.annotation.PreDestroy;
 import javax.enterprise.context.ApplicationScoped;
 import javax.enterprise.event.Observes;
-import javax.enterprise.inject.Instance;
 import javax.inject.Inject;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.charset.Charset;
@@ -101,7 +101,7 @@ public class HttpClientServiceImpl implements HttpClientService {
 
     @Inject
     @MarmottaCache("http-client-cache")
-    private Instance<Ehcache>            ehcache;
+    private Cache httpCache;
 
     private HttpClient                   httpClient;
     private IdleConnectionMonitorThread  idleConnectionMonitorThread;
@@ -308,7 +308,7 @@ public class HttpClientServiceImpl implements HttpClientService {
                 cacheConfig.setMaxCacheEntries(1000);
                 cacheConfig.setMaxObjectSize(81920);
 
-                final HttpCacheStorage cacheStore = new EhcacheHttpCacheStorage(ehcache.get(), cacheConfig);
+                final HttpCacheStorage cacheStore = new InfinispanHttpCacheStorage(httpCache);
 
                 this.httpClient = new MonitoredHttpClient(new CachingHttpClient(hc, cacheStore, cacheConfig));
             } else {
@@ -806,4 +806,101 @@ public class HttpClientServiceImpl implements HttpClientService {
         }
 
     }
+
+
+
+    private static class InfinispanHttpCacheStorage implements HttpCacheStorage {
+
+        Cache<String, byte[]> cache;
+
+        private final HttpCacheEntrySerializer serializer;
+
+
+        private InfinispanHttpCacheStorage(Cache<String, byte[]> cache) {
+            this.cache      = cache;
+            this.serializer = new DefaultHttpCacheEntrySerializer();
+        }
+
+        /**
+         * Store a given cache entry under the given key.
+         *
+         * @param key   where in the cache to store the entry
+         * @param entry cached response to store
+         * @throws java.io.IOException
+         */
+        @Override
+        public void putEntry(String key, HttpCacheEntry entry) throws IOException {
+            final ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            serializer.writeTo(entry, bos);
+
+            cache.put(key,bos.toByteArray());
+        }
+
+        /**
+         * Retrieves the cache entry stored under the given key
+         * or null if no entry exists under that key.
+         *
+         * @param key cache key
+         * @return an {@link org.apache.http.client.cache.HttpCacheEntry} or {@code null} if no
+         * entry exists
+         * @throws java.io.IOException
+         */
+        @Override
+        public HttpCacheEntry getEntry(String key) throws IOException {
+            byte[] data = cache.get(key);
+            if(data == null) {
+                return null;
+            } else {
+                return serializer.readFrom(new ByteArrayInputStream(data));
+            }
+        }
+
+        /**
+         * Deletes/invalidates/removes any cache entries currently
+         * stored under the given key.
+         *
+         * @param key
+         * @throws java.io.IOException
+         */
+        @Override
+        public void removeEntry(String key) throws IOException {
+            cache.remove(key);
+        }
+
+        /**
+         * Atomically applies the given callback to update an existing cache
+         * entry under a given key.
+         *
+         * @param key      indicates which entry to modify
+         * @param callback performs the update; see
+         *                 {@link org.apache.http.client.cache.HttpCacheUpdateCallback} for details, but roughly the
+         *                 callback expects to be handed the current entry and will return
+         *                 the new value for the entry.
+         * @throws java.io.IOException
+         * @throws org.apache.http.client.cache.HttpCacheUpdateException
+         */
+        @Override
+        public void updateEntry(String key, HttpCacheUpdateCallback callback) throws IOException, HttpCacheUpdateException {
+            final byte[] oldData = cache.get(key);
+
+            HttpCacheEntry existingEntry = null;
+            if(oldData != null){
+                existingEntry = serializer.readFrom(new ByteArrayInputStream(oldData));
+            }
+
+            final HttpCacheEntry updatedEntry = callback.update(existingEntry);
+
+            if (existingEntry == null) {
+                putEntry(key, updatedEntry);
+                return;
+            } else {
+                // Attempt to do a CAS replace, if we fail then retry
+                // While this operation should work fine within this instance, multiple instances
+                //  could trample each others' data
+                final ByteArrayOutputStream bos = new ByteArrayOutputStream();
+                serializer.writeTo(updatedEntry, bos);
+                cache.replace(key, oldData, bos.toByteArray());
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/modules/MarmottaResourceServiceImpl.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/modules/MarmottaResourceServiceImpl.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/modules/MarmottaResourceServiceImpl.java
index 1aecba2..0b837e1 100644
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/modules/MarmottaResourceServiceImpl.java
+++ b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/modules/MarmottaResourceServiceImpl.java
@@ -27,15 +27,13 @@ import org.apache.marmotta.platform.core.api.modules.ResourceEntry;
 import org.apache.marmotta.platform.core.events.SystemStartupEvent;
 import org.apache.marmotta.platform.core.model.module.ModuleConfiguration;
 import org.apache.marmotta.platform.core.qualifiers.cache.MarmottaCache;
-import net.sf.ehcache.Ehcache;
-import net.sf.ehcache.Element;
+import org.infinispan.Cache;
 import org.slf4j.Logger;
 
 import javax.annotation.PostConstruct;
 import javax.enterprise.context.ApplicationScoped;
 import javax.enterprise.event.Observes;
 import javax.inject.Inject;
-
 import java.io.IOException;
 import java.net.URL;
 import java.util.Collection;
@@ -66,7 +64,7 @@ public class MarmottaResourceServiceImpl implements MarmottaResourceService {
 
 
     @Inject @MarmottaCache("resource-cache")
-    private Ehcache resourceCache;
+    private Cache resourceCache;
 
     /**
      * Used for detecting the mime type of resources contained in KiWi modules
@@ -219,12 +217,12 @@ public class MarmottaResourceServiceImpl implements MarmottaResourceService {
 
 
     private boolean isCached(String key) {
-        return isCacheEnabled() && resourceCache.isKeyInCache(key) && resourceCache.get(key) != null;
+        return isCacheEnabled() && resourceCache.containsKey(key) && resourceCache.get(key) != null;
     }
 
     private ResourceEntry getFromCache(String key) {
         if (isCacheEnabled())
-            return (ResourceEntry) resourceCache.get(key).getObjectValue();
+            return (ResourceEntry) resourceCache.get(key);
         else
             return null;
     }
@@ -232,7 +230,7 @@ public class MarmottaResourceServiceImpl implements MarmottaResourceService {
     // Store in the cache
     private void putInCache(String key, ResourceEntry data) {
         if(isCacheEnabled()) {
-            resourceCache.put(new Element(key,data));
+            resourceCache.put(key,data);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-core/src/main/resources/config-defaults.properties
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/resources/config-defaults.properties b/platform/marmotta-core/src/main/resources/config-defaults.properties
index 3933aae..278dd9a 100644
--- a/platform/marmotta-core/src/main/resources/config-defaults.properties
+++ b/platform/marmotta-core/src/main/resources/config-defaults.properties
@@ -192,3 +192,13 @@ prefix.rdf = http://www.w3.org/1999/02/22-rdf-syntax-ns#
 prefix.skos = http://www.w3.org/2004/02/skos/core#
 prefix.ldp = http://www.w3.org/ns/ldp#
 prefix.mao = http://www.w3.org/ns/ma-ont#
+
+
+###############################################################################
+# Clustering Configuration
+###############################################################################
+
+
+# Turn on cluster-specific configuration options (e.g. replicated and distributed caching, synchronization, ...)
+clustering.enabled = false
+clustering.name    = Marmotta
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-core/src/main/resources/config-descriptions.properties
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/resources/config-descriptions.properties b/platform/marmotta-core/src/main/resources/config-descriptions.properties
index 213fb6c..43983c0 100644
--- a/platform/marmotta-core/src/main/resources/config-descriptions.properties
+++ b/platform/marmotta-core/src/main/resources/config-descriptions.properties
@@ -158,3 +158,10 @@ importer.batchsize.type = java.lang.Integer(10|0|*)
 statistics.enabled.description = true
 statistics.enabled.type = java.lang.Boolean
 
+
+
+clustering.enabled.description = Turn on cluster-specific configuration options (e.g. replicated and distributed caching, synchronization, ...)
+clustering.enabled.type = java.lang.Boolean
+
+clustering.name.description = Cluster name to use in cluster configuration (e.g. cache cluster name)
+clustering.name.type = java.lang.String
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-core/src/main/resources/jgroups-marmotta.xml
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/resources/jgroups-marmotta.xml b/platform/marmotta-core/src/main/resources/jgroups-marmotta.xml
new file mode 100644
index 0000000..aa5ce8c
--- /dev/null
+++ b/platform/marmotta-core/src/main/resources/jgroups-marmotta.xml
@@ -0,0 +1,74 @@
+<config xmlns="urn:org:jgroups"
+        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+        xsi:schemaLocation="urn:org:jgroups http://www.jgroups.org/schema/JGroups-3.4.xsd">
+   <UDP
+         mcast_addr="${jgroups.udp.mcast_addr:228.6.7.8}"
+         mcast_port="${jgroups.udp.mcast_port:46655}"
+         tos="8"
+         ucast_recv_buf_size="20m"
+         ucast_send_buf_size="640k"
+         mcast_recv_buf_size="25m"
+         mcast_send_buf_size="640k"
+         loopback="true"
+         max_bundle_size="31k"
+         ip_ttl="${jgroups.udp.ip_ttl:2}"
+         enable_diagnostics="false"
+         bundler_type="old"
+
+         thread_naming_pattern="pl"
+
+         thread_pool.enabled="true"
+         thread_pool.min_threads="2"
+         thread_pool.max_threads="30"
+         thread_pool.keep_alive_time="60000"
+         thread_pool.queue_enabled="true"
+         thread_pool.queue_max_size="100"
+         thread_pool.rejection_policy="Discard"
+
+         oob_thread_pool.enabled="true"
+         oob_thread_pool.min_threads="2"
+         oob_thread_pool.max_threads="30"
+         oob_thread_pool.keep_alive_time="60000"
+         oob_thread_pool.queue_enabled="false"
+         oob_thread_pool.queue_max_size="100"
+         oob_thread_pool.rejection_policy="Discard"
+
+         internal_thread_pool.enabled="true"
+         internal_thread_pool.min_threads="1"
+         internal_thread_pool.max_threads="10"
+         internal_thread_pool.keep_alive_time="60000"
+         internal_thread_pool.queue_enabled="true"
+         internal_thread_pool.queue_max_size="100"
+         internal_thread_pool.rejection_policy="Discard"
+         />
+
+   <PING timeout="3000" num_initial_members="3"/>
+   <MERGE2 max_interval="30000" min_interval="10000"/>
+
+   <FD_SOCK/>
+   <FD_ALL timeout="15000" interval="3000"/>
+   <VERIFY_SUSPECT timeout="1500"/>
+
+   <pbcast.NAKACK2
+                    xmit_interval="1000"
+                    xmit_table_num_rows="100"
+                    xmit_table_msgs_per_row="10000"
+                    xmit_table_max_compaction_time="10000"
+                    max_msg_batch_size="100"/>
+   <UNICAST3
+              xmit_interval="500"
+              xmit_table_num_rows="20"
+              xmit_table_msgs_per_row="10000"
+              xmit_table_max_compaction_time="10000"
+              max_msg_batch_size="100"
+              conn_expiry_timeout="0"/>
+
+   <pbcast.STABLE stability_delay="500" desired_avg_gossip="5000" max_bytes="1m"/>
+   <pbcast.GMS print_local_addr="false" join_timeout="3000" view_bundling="true"/>
+   <tom.TOA/> <!-- the TOA is only needed for total order transactions-->
+
+   <UFC max_credits="2m" min_threshold="0.40"/>
+   <MFC max_credits="2m" min_threshold="0.40"/>
+   <FRAG2 frag_size="30k"  />
+   <RSVP timeout="60000" resend_interval="500" ack_on_delivery="false" />
+</config>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/61c14a19/platform/marmotta-user/src/main/java/org/apache/marmotta/platform/user/services/AccountServiceImpl.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-user/src/main/java/org/apache/marmotta/platform/user/services/AccountServiceImpl.java b/platform/marmotta-user/src/main/java/org/apache/marmotta/platform/user/services/AccountServiceImpl.java
index f0ab876..beede92 100644
--- a/platform/marmotta-user/src/main/java/org/apache/marmotta/platform/user/services/AccountServiceImpl.java
+++ b/platform/marmotta-user/src/main/java/org/apache/marmotta/platform/user/services/AccountServiceImpl.java
@@ -18,8 +18,6 @@
 package org.apache.marmotta.platform.user.services;
 
 import com.google.common.base.Preconditions;
-import net.sf.ehcache.Ehcache;
-import net.sf.ehcache.Element;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.marmotta.commons.sesame.model.Namespaces;
 import org.apache.marmotta.platform.core.api.config.ConfigurationService;
@@ -32,6 +30,7 @@ import org.apache.marmotta.platform.core.qualifiers.cache.MarmottaCache;
 import org.apache.marmotta.platform.user.api.AccountService;
 import org.apache.marmotta.platform.user.model.UserAccount;
 import org.apache.marmotta.platform.user.model.UserAccount.PasswordHash;
+import org.infinispan.Cache;
 import org.openrdf.model.Resource;
 import org.openrdf.model.URI;
 import org.slf4j.Logger;
@@ -59,7 +58,7 @@ public class AccountServiceImpl implements AccountService {
 
     @Inject
     @MarmottaCache("user-cache")
-    private Ehcache              userCache;
+    private Cache<String,UserAccount>  userCache;
 
     private PasswordHash         hashAlgo;
 
@@ -120,8 +119,8 @@ public class AccountServiceImpl implements AccountService {
 
 
         for (UserAccount userAccount : list) {
-            userCache.put(new Element(userAccount.getLogin(), userAccount));
-            userCache.put(new Element(userAccount.getWebId(), userAccount));
+            userCache.put(userAccount.getLogin(), userAccount);
+            userCache.put(userAccount.getWebId(), userAccount);
         }
         return list;
     }
@@ -185,7 +184,7 @@ public class AccountServiceImpl implements AccountService {
         if (StringUtils.isBlank(login)) return null;
         UserAccount account = null;
         if (userCache != null && userCache.get(login) != null) {
-            account = (UserAccount) userCache.get(login).getObjectValue();
+            account = (UserAccount) userCache.get(login);
         } else {
             if (configurationService.isConfigurationSet("user."+login+".webid")) {
                 account = new UserAccount();
@@ -195,8 +194,8 @@ public class AccountServiceImpl implements AccountService {
                 account.setRoles(new HashSet<String>(configurationService.getListConfiguration("user."+login+".roles")));
                 account.setWebId(configurationService.getStringConfiguration("user."+login+".webid"));
 
-                userCache.put(new Element(account.getLogin(), account));
-                userCache.put(new Element(account.getWebId(), account));
+                userCache.put(account.getLogin(), account);
+                userCache.put(account.getWebId(), account);
             } else {
                 log.info("UserAccount {} not found", login);
             }
@@ -210,7 +209,7 @@ public class AccountServiceImpl implements AccountService {
 
         UserAccount account = null;
         if (userCache != null && userCache.get(resource) != null) {
-            account = (UserAccount) userCache.get(resource).getObjectValue();
+            account = userCache.get(resource);
         } else {
             for(UserAccount a : listAccounts()) {
                 if(a.getWebId().equals(resource.stringValue())) {
@@ -219,8 +218,8 @@ public class AccountServiceImpl implements AccountService {
                 }
             }
             if (account != null) {
-                userCache.put(new Element(account.getLogin(), account));
-                userCache.put(new Element(account.getWebId(), account));
+                userCache.put(account.getLogin(), account);
+                userCache.put(account.getWebId(), account);
             } else {
                 log.warn("UserAccount {} not found", resource);
             }


[3/7] git commit: towards MARMOTTA-388: cleanup references to ehcache

Posted by ss...@apache.org.
towards MARMOTTA-388: cleanup references to ehcache


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

Branch: refs/heads/develop
Commit: af9a8002d13e22a5b1d9897aad735609c0133bb2
Parents: 0d79835
Author: Sebastian Schaffert <ss...@apache.org>
Authored: Mon Dec 16 20:06:35 2013 +0100
Committer: Sebastian Schaffert <ss...@apache.org>
Committed: Mon Dec 16 20:06:35 2013 +0100

----------------------------------------------------------------------
 commons/sesame-transactions/pom.xml             |   5 -
 .../kiwi/loader/generic/Statistics.java         |  17 +-
 .../persistence/KiWiReasoningConnection.java    |  42 +-
 .../marmotta/kiwi/caching/KiWiCacheManager.java |   7 +-
 .../ldcache/ldcache-backend-ehcache/pom.xml     |  78 --
 .../src/main/resources/ehcache-ldcache.xml      | 742 -------------------
 libraries/ldcache/ldcache-backend-mapdb/pom.xml |  39 -
 libraries/ldcache/pom.xml                       |   1 -
 parent/pom.xml                                  |   6 -
 .../backend/kiwi/KiWiStoreProvider.java         |   1 +
 .../main/resources/config-defaults.properties   |   4 +
 .../resources/config-descriptions.properties    |   3 +
 platform/marmotta-core/pom.xml                  |   8 +-
 13 files changed, 33 insertions(+), 920 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/commons/sesame-transactions/pom.xml
----------------------------------------------------------------------
diff --git a/commons/sesame-transactions/pom.xml b/commons/sesame-transactions/pom.xml
index 1d3aef6..20d4dfd 100644
--- a/commons/sesame-transactions/pom.xml
+++ b/commons/sesame-transactions/pom.xml
@@ -83,11 +83,6 @@
             <scope>test</scope>
         </dependency>
         <dependency>
-            <groupId>net.sf.ehcache</groupId>
-            <artifactId>ehcache-core</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
             <artifactId>hamcrest-core</artifactId>
             <groupId>org.hamcrest</groupId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/Statistics.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/Statistics.java b/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/Statistics.java
index 45d26d1..1c2353a 100644
--- a/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/Statistics.java
+++ b/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/Statistics.java
@@ -17,15 +17,10 @@
 
 package org.apache.marmotta.kiwi.loader.generic;
 
-import net.sf.ehcache.constructs.blocking.SelfPopulatingCache;
 import org.apache.marmotta.kiwi.loader.util.UnitFormatter;
 import org.rrd4j.ConsolFun;
 import org.rrd4j.DsType;
-import org.rrd4j.core.FetchData;
-import org.rrd4j.core.FetchRequest;
-import org.rrd4j.core.RrdDb;
-import org.rrd4j.core.RrdDef;
-import org.rrd4j.core.Sample;
+import org.rrd4j.core.*;
 import org.rrd4j.graph.RrdGraph;
 import org.rrd4j.graph.RrdGraphDef;
 import org.slf4j.Logger;
@@ -85,8 +80,6 @@ public class Statistics {
         stCfg.addDatasource("triples", DsType.COUNTER, 600, Double.NaN, Double.NaN);
         stCfg.addDatasource("nodes", DsType.COUNTER, 600, Double.NaN, Double.NaN);
         stCfg.addDatasource("nodes-loaded", DsType.COUNTER, 600, Double.NaN, Double.NaN);
-        stCfg.addDatasource("cache-hits", DsType.COUNTER, 600, Double.NaN, Double.NaN);
-        stCfg.addDatasource("cache-misses", DsType.COUNTER, 600, Double.NaN, Double.NaN);
         stCfg.addArchive(ConsolFun.AVERAGE, 0.5, 1, 1440);  // every five seconds for 2 hours
         stCfg.addArchive(ConsolFun.AVERAGE, 0.5, 12, 1440); // every minute for 1 day
         stCfg.addArchive(ConsolFun.AVERAGE, 0.5, 60, 1440); // every five minutes for five days
@@ -158,18 +151,12 @@ public class Statistics {
         @Override
         public void run() {
 
-            long cacheMisses = 0, cacheHits = 0;
-            for(SelfPopulatingCache c : new SelfPopulatingCache[] { handler.literalCache, handler.uriCache, handler.bnodeCache }) {
-                cacheHits   += c.getStatistics().getCacheHits();
-                cacheMisses += c.getStatistics().getCacheMisses();
-            }
-
             try {
                 long time = System.currentTimeMillis() / 1000;
 
                 synchronized (statSample) {
                     statSample.setTime(time);
-                    statSample.setValues(handler.triples, handler.nodes, handler.nodesLoaded, cacheHits, cacheMisses);
+                    statSample.setValues(handler.triples, handler.nodes, handler.nodesLoaded);
                     statSample.update();
                 }
 

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/libraries/kiwi/kiwi-reasoner/src/main/java/org/apache/marmotta/kiwi/reasoner/persistence/KiWiReasoningConnection.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-reasoner/src/main/java/org/apache/marmotta/kiwi/reasoner/persistence/KiWiReasoningConnection.java b/libraries/kiwi/kiwi-reasoner/src/main/java/org/apache/marmotta/kiwi/reasoner/persistence/KiWiReasoningConnection.java
index 6de7eba..74acc13 100644
--- a/libraries/kiwi/kiwi-reasoner/src/main/java/org/apache/marmotta/kiwi/reasoner/persistence/KiWiReasoningConnection.java
+++ b/libraries/kiwi/kiwi-reasoner/src/main/java/org/apache/marmotta/kiwi/reasoner/persistence/KiWiReasoningConnection.java
@@ -17,13 +17,7 @@
  */
 package org.apache.marmotta.kiwi.reasoner.persistence;
 
-import info.aduna.iteration.CloseableIteration;
-import info.aduna.iteration.EmptyIteration;
-import info.aduna.iteration.Iteration;
-import info.aduna.iteration.Iterations;
-import info.aduna.iteration.IteratorIteration;
-import net.sf.ehcache.Cache;
-import net.sf.ehcache.Element;
+import info.aduna.iteration.*;
 import org.apache.marmotta.kiwi.caching.KiWiCacheManager;
 import org.apache.marmotta.kiwi.model.rdf.KiWiNode;
 import org.apache.marmotta.kiwi.model.rdf.KiWiTriple;
@@ -32,18 +26,11 @@ import org.apache.marmotta.kiwi.persistence.KiWiDialect;
 import org.apache.marmotta.kiwi.persistence.KiWiPersistence;
 import org.apache.marmotta.kiwi.persistence.util.ResultSetIteration;
 import org.apache.marmotta.kiwi.persistence.util.ResultTransformerFunction;
-import org.apache.marmotta.kiwi.reasoner.model.program.Field;
-import org.apache.marmotta.kiwi.reasoner.model.program.Filter;
-import org.apache.marmotta.kiwi.reasoner.model.program.Justification;
-import org.apache.marmotta.kiwi.reasoner.model.program.LiteralField;
-import org.apache.marmotta.kiwi.reasoner.model.program.Pattern;
-import org.apache.marmotta.kiwi.reasoner.model.program.Program;
-import org.apache.marmotta.kiwi.reasoner.model.program.ResourceField;
-import org.apache.marmotta.kiwi.reasoner.model.program.Rule;
-import org.apache.marmotta.kiwi.reasoner.model.program.VariableField;
+import org.apache.marmotta.kiwi.reasoner.model.program.*;
 import org.apache.marmotta.kiwi.reasoner.model.query.QueryResult;
 import org.apache.marmotta.kiwi.reasoner.parser.KWRLProgramParser;
 import org.apache.marmotta.kiwi.reasoner.parser.ParseException;
+import org.infinispan.Cache;
 import org.openrdf.model.ValueFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,14 +39,7 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 
 /**
  * Extends the basic KiWiConnection by functionalities for storing, deleting and querying reasoner programs and rules
@@ -73,7 +53,7 @@ public class KiWiReasoningConnection extends KiWiConnection {
 
     private ValueFactory valueFactory;
 
-    private Cache ruleIdCache;
+    private Cache<Long,Rule> ruleIdCache;
 
     public KiWiReasoningConnection(KiWiPersistence persistence, KiWiDialect dialect, KiWiCacheManager cacheManager, ValueFactory valueFactory) throws SQLException {
         super(persistence, dialect, cacheManager);
@@ -110,7 +90,7 @@ public class KiWiReasoningConnection extends KiWiConnection {
             insertRule.executeUpdate();
         }
 
-        ruleIdCache.put(new Element(rule.getId(), rule));
+        ruleIdCache.put(rule.getId(), rule);
     }
 
     /**
@@ -122,10 +102,10 @@ public class KiWiReasoningConnection extends KiWiConnection {
      */
     public Rule loadRuleById(long ruleId, Map<String, String> namespaces) throws SQLException {
 
-        Element cached = ruleIdCache.get(ruleId);
+        Rule cached = ruleIdCache.get(ruleId);
 
         if(cached != null) {
-            return (Rule) cached.getObjectValue();
+            return cached;
         } else {
             requireJDBCConnection();
 
@@ -192,10 +172,10 @@ public class KiWiReasoningConnection extends KiWiConnection {
 
 
     protected Rule constructRuleFromDatabase(ResultSet row, Map<String, String> namespaces) throws SQLException, ParseException {
-        Element cached = ruleIdCache.get(row.getLong("id"));
+        Rule cached = ruleIdCache.get(row.getLong("id"));
 
         if(cached != null) {
-            return (Rule) cached.getObjectValue();
+            return cached;
         } else {
 
             Rule result = KWRLProgramParser.parseRule(row.getString("body"),namespaces, valueFactory);
@@ -203,7 +183,7 @@ public class KiWiReasoningConnection extends KiWiConnection {
             result.setName(row.getString("name"));
             result.setDescription(row.getString("description"));
 
-            ruleIdCache.put(new Element(result.getId(), result));
+            ruleIdCache.put(result.getId(), result);
 
             return result;
         }

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
index 7be150c..8af3ba6 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
@@ -87,7 +87,7 @@ public class KiWiCacheManager {
                             .consistentHashFactory(new SyncConsistentHashFactory())
                     .eviction()
                         .strategy(EvictionStrategy.LIRS)
-                        .maxEntries(100000)
+                        .maxEntries(1000)
                     .expiration()
                         .lifespan(5, TimeUnit.MINUTES)
                         .maxIdle(1, TimeUnit.MINUTES)
@@ -141,6 +141,8 @@ public class KiWiCacheManager {
     public Cache getTripleCache() {
         if(!cacheManager.cacheExists(TRIPLE_CACHE)) {
             Configuration tripleConfiguration = new ConfigurationBuilder().read(defaultConfiguration)
+                    .eviction()
+                        .maxEntries(100000)
                     .expiration()
                         .lifespan(60, TimeUnit.SECONDS)
                         .maxIdle(30, TimeUnit.SECONDS)
@@ -298,6 +300,9 @@ public class KiWiCacheManager {
      * @return
      */
     public synchronized Cache getCacheByName(String name) {
+        if(!cacheManager.cacheExists(name)) {
+            cacheManager.defineConfiguration(name, new ConfigurationBuilder().read(defaultConfiguration).build());
+        }
         return cacheManager.getCache(name);
 
     }

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/libraries/ldcache/ldcache-backend-ehcache/pom.xml
----------------------------------------------------------------------
diff --git a/libraries/ldcache/ldcache-backend-ehcache/pom.xml b/libraries/ldcache/ldcache-backend-ehcache/pom.xml
deleted file mode 100644
index 54c0f95..0000000
--- a/libraries/ldcache/ldcache-backend-ehcache/pom.xml
+++ /dev/null
@@ -1,78 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-   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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.marmotta</groupId>
-        <artifactId>marmotta-parent</artifactId>
-        <version>3.2.0-SNAPSHOT</version>
-        <relativePath>../../../parent</relativePath>
-    </parent>
-
-    <artifactId>ldcache-backend-ehcache</artifactId>
-    <name>LDCache Backend: EHCache</name>
-
-    <description>
-        This module offers a backend that allows storing cache entries in an EHCache instance. The backend class is
-        abstract and needs to be subclassed to provide the repository connection for the triples.
-    </description>
-
-
-    <build>
-        <pluginManagement>
-            <plugins>
-                <plugin> <!-- generate JRebel Configuration -->
-                    <groupId>org.zeroturnaround</groupId>
-                    <artifactId>jrebel-maven-plugin</artifactId>
-                    <executions>
-                        <execution>
-                            <id>generate-rebel-xml</id>
-                            <phase>process-resources</phase>
-                            <goals>
-                                <goal>generate</goal>
-                            </goals>
-                        </execution>
-                    </executions>
-                    <configuration>
-                        <relativePath>../../../</relativePath>
-                        <rootPath>$${rebel.root}</rootPath>
-                    </configuration>
-                </plugin>
-            </plugins>
-        </pluginManagement>
-    </build>
-
-    <dependencies>
-
-
-        <dependency>
-            <groupId>org.apache.marmotta</groupId>
-            <artifactId>ldcache-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.marmotta</groupId>
-            <artifactId>sesame-tripletable</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>net.sf.ehcache</groupId>
-            <artifactId>ehcache-core</artifactId>
-        </dependency>
-    </dependencies>
-    
-</project>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/libraries/ldcache/ldcache-backend-ehcache/src/main/resources/ehcache-ldcache.xml
----------------------------------------------------------------------
diff --git a/libraries/ldcache/ldcache-backend-ehcache/src/main/resources/ehcache-ldcache.xml b/libraries/ldcache/ldcache-backend-ehcache/src/main/resources/ehcache-ldcache.xml
deleted file mode 100644
index 590eb26..0000000
--- a/libraries/ldcache/ldcache-backend-ehcache/src/main/resources/ehcache-ldcache.xml
+++ /dev/null
@@ -1,742 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-
-    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.
-
--->
-<!--
-CacheManager Configuration
-==========================
-An ehcache.xml corresponds to a single CacheManager.
-
-See instructions below or the ehcache schema (ehcache.xsd) on how to configure.
-
-System property tokens can be specified in this file which are replaced when the configuration
-is loaded. For example multicastGroupPort=${multicastGroupPort} can be replaced with the
-System property either from an environment variable or a system property specified with a
-command line switch such as -DmulticastGroupPort=4446. Another example, useful for Terracotta
-server based deployments is <terracottaConfig url="${serverAndPort}"/ and specify a command line
-switch of -Dserver36:9510
-
-The attributes of <ehcache> are:
-* name - an optional name for the CacheManager.  The name is optional and primarily used
-for documentation or to distinguish Terracotta clustered cache state.  With Terracotta
-clustered caches, a combination of CacheManager name and cache name uniquely identify a
-particular cache store in the Terracotta clustered memory.
-* updateCheck - an optional boolean flag specifying whether this CacheManager should check
-for new versions of Ehcache over the Internet.  If not specified, updateCheck="true".
-* dynamicConfig - an optional setting that can be used to disable dynamic configuration of caches
-associated with this CacheManager.  By default this is set to true - i.e. dynamic configuration
-is enabled.  Dynamically configurable caches can have their TTI, TTL and maximum disk and
-in-memory capacity changed at runtime through the cache's configuration object.
-* monitoring - an optional setting that determines whether the CacheManager should
-automatically register the SampledCacheMBean with the system MBean server.
-
-Currently, this monitoring is only useful when using Terracotta clustering and using the
-Terracotta Developer Console. With the "autodetect" value, the presence of Terracotta clustering
-will be detected and monitoring, via the Developer Console, will be enabled. Other allowed values
-are "on" and "off".  The default is "autodetect". This setting does not perform any function when
-used with JMX monitors.
-
-* maxBytesLocalHeap - optional setting that constraints the memory usage of the Caches managed by the CacheManager
-to use at most the specified number of bytes of the local VM's heap.
-* maxBytesLocalOffHeap - optional setting that constraints the offHeap usage of the Caches managed by the CacheManager
-to use at most the specified number of bytes of the local VM's offHeap memory.
-* maxBytesLocalDisk - optional setting that constraints the disk usage of the Caches managed by the CacheManager
-to use at most the specified number of bytes of the local disk.
-
-These settings let you define "resource pools", caches will share. For instance setting maxBytesLocalHeap to 100M, will result in
-all caches sharing 100 MegaBytes of ram. The CacheManager will balance these 100 MB across all caches based on their respective usage
-patterns. You can allocate a precise amount of bytes to a particular cache by setting the appropriate maxBytes* attribute for that cache.
-That amount will be subtracted from the CacheManager pools, so that if a cache a specified 30M requirement, the other caches will share
-the remaining 70M.
-
-Also, specifying a maxBytesLocalOffHeap at the CacheManager level will result in overflowToOffHeap to be true by default. If you don't want
-a specific cache to overflow to off heap, you'll have to set overflowToOffHeap="false" explicitly
-
-Here is an example of CacheManager level resource tuning, which will use up to 400M of heap and 2G of offHeap:
-
-<ehcache xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:noNamespaceSchemaLocation="ehcache.xsd"
-         updateCheck="true" monitoring="autodetect"
-         dynamicConfig="true" maxBytesLocalHeap="400M" maxBytesLocalOffHeap="2G">
-
--->
-<ehcache name="LDCache"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:noNamespaceSchemaLocation="http://ehcache.org/ehcache.xsd"
-         updateCheck="true" monitoring="autodetect"
-         dynamicConfig="true">
-
-
-
-    <!--
-    DiskStore configuration
-    =======================
-
-    The diskStore element is optional. To turn off disk store path creation, comment out the diskStore
-    element below.
-
-    Configure it if you have disk persistence enabled for any cache or if you use
-    unclustered indexed search.
-
-    If it is not configured, and a cache is created which requires a disk store, a warning will be
-     issued and java.io.tmpdir will automatically be used.
-
-    diskStore has only one attribute - "path". It is the path to the directory where
-    any required disk files will be created.
-
-    If the path is one of the following Java System Property it is replaced by its value in the
-    running VM. For backward compatibility these should be specified without being enclosed in the ${token}
-    replacement syntax.
-
-    The following properties are translated:
-    * user.home - User's home directory
-    * user.dir - User's current working directory
-    * java.io.tmpdir - Default temp file path
-    * ehcache.disk.store.dir - A system property you would normally specify on the command line
-      e.g. java -Dehcache.disk.store.dir=/u01/myapp/diskdir ...
-
-    Subdirectories can be specified below the property e.g. java.io.tmpdir/one
-
-    -->
-    <diskStore path="java.io.tmpdir"/>
-
-
-    <!--
-    Cache configuration
-    ===================
-
-    The following attributes are required.
-
-    name:
-    Sets the name of the cache. This is used to identify the cache. It must be unique.
-
-    maxEntriesLocalHeap:
-    Sets the maximum number of objects that will be created in memory.  0 = no limit.
-    In practice no limit means Integer.MAX_SIZE (2147483647) unless the cache is distributed
-    with a Terracotta server in which case it is limited by resources.
-
-    maxEntriesLocalDisk:
-    Sets the maximum number of objects that will be maintained in the DiskStore
-    The default value is zero, meaning unlimited.
-
-    eternal:
-    Sets whether elements are eternal. If eternal,  timeouts are ignored and the
-    element is never expired.
-
-    The following attributes and elements are optional.
-
-    overflowToOffHeap:
-    (boolean) This feature is available only in enterprise versions of Ehcache.
-    When set to true, enables the cache to utilize off-heap memory
-    storage to improve performance. Off-heap memory is not subject to Java
-    GC. The default value is false.
-
-    maxBytesLocalHeap:
-    Defines how many bytes the cache may use from the VM's heap. If a CacheManager
-    maxBytesLocalHeap has been defined, this Cache's specified amount will be
-    subtracted from the CacheManager. Other caches will share the remainder.
-    This attribute's values are given as <number>k|K|m|M|g|G for
-    kilobytes (k|K), megabytes (m|M), or gigabytes (g|G).
-    For example, maxBytesLocalHeap="2g" allots 2 gigabytes of heap memory.
-    If you specify a maxBytesLocalHeap, you can't use the maxEntriesLocalHeap attribute.
-    maxEntriesLocalHeap can't be used if a CacheManager maxBytesLocalHeap is set.
-
-    Elements put into the cache will be measured in size using net.sf.ehcache.pool.sizeof.SizeOf
-    If you wish to ignore some part of the object graph, see net.sf.ehcache.pool.sizeof.annotations.IgnoreSizeOf
-
-    maxBytesLocalOffHeap:
-    This feature is available only in enterprise versions of Ehcache.
-    Sets the amount of off-heap memory this cache can use, and will reserve.
-
-    This setting will set overflowToOffHeap to true. Set explicitly to false to disable overflow behavior.
-
-    Note that it is recommended to set maxEntriesLocalHeap to at least 100 elements
-    when using an off-heap store, otherwise performance will be seriously degraded,
-    and a warning will be logged.
-
-    The minimum amount that can be allocated is 128MB. There is no maximum.
-
-    maxBytesLocalDisk:
-    As for maxBytesLocalHeap, but specifies the limit of disk storage this cache will ever use.
-
-    timeToIdleSeconds:
-    Sets the time to idle for an element before it expires.
-    i.e. The maximum amount of time between accesses before an element expires
-    Is only used if the element is not eternal.
-    Optional attribute. A value of 0 means that an Element can idle for infinity.
-    The default value is 0.
-
-    timeToLiveSeconds:
-    Sets the time to live for an element before it expires.
-    i.e. The maximum time between creation time and when an element expires.
-    Is only used if the element is not eternal.
-    Optional attribute. A value of 0 means that and Element can live for infinity.
-    The default value is 0.
-
-    diskExpiryThreadIntervalSeconds:
-    The number of seconds between runs of the disk expiry thread. The default value
-    is 120 seconds.
-
-    diskSpoolBufferSizeMB:
-    This is the size to allocate the DiskStore for a spool buffer. Writes are made
-    to this area and then asynchronously written to disk. The default size is 30MB.
-    Each spool buffer is used only by its cache. If you get OutOfMemory errors consider
-    lowering this value. To improve DiskStore performance consider increasing it. Trace level
-    logging in the DiskStore will show if put back ups are occurring.
-
-    clearOnFlush:
-    whether the MemoryStore should be cleared when flush() is called on the cache.
-    By default, this is true i.e. the MemoryStore is cleared.
-
-    statistics:
-    Whether to collect statistics. Note that this should be turned on if you are using
-    the Ehcache Monitor. By default statistics is turned off to favour raw performance.
-    To enable set statistics="true"
-
-    memoryStoreEvictionPolicy:
-    Policy would be enforced upon reaching the maxEntriesLocalHeap limit. Default
-    policy is Least Recently Used (specified as LRU). Other policies available -
-    First In First Out (specified as FIFO) and Less Frequently Used
-    (specified as LFU)
-
-    copyOnRead:
-    Whether an Element is copied when being read from a cache.
-    By default this is false.
-
-    copyOnWrite:
-    Whether an Element is copied when being added to the cache.
-    By default this is false.
-
-    Cache persistence is configured through the persistence sub-element.  The attributes of the
-    persistence element are:
-
-    strategy:
-    Configures the type of persistence provided by the configured cache.  This must be one of the
-    following values:
-
-    * localRestartable - Enables the RestartStore and copies all cache entries (on-heap and/or off-heap)
-    to disk. This option provides fast restartability with fault tolerant cache persistence on disk.
-    It is available for Enterprise Ehcache users only.
-
-    * localTempSwap - Swaps cache entries (on-heap and/or off-heap) to disk when the cache is full.
-    "localTempSwap" is not persistent.
-
-    * none - Does not persist cache entries.
-
-    * distributed - Defers to the <terracotta> configuration for persistence settings. This option
-    is not applicable for standalone.
-
-    synchronousWrites:
-    When set to true write operations on the cache do not return until after the operations data has been
-    successfully flushed to the disk storage.  This option is only valid when used with the "localRestartable"
-    strategy, and defaults to false.
-
-    The following example configuration shows a cache configured for localTempSwap restartability.
-
-    <cache name="persistentCache" maxEntriesLocalHeap="1000">
-        <persistence strategy="localTempSwap"/>
-    </cache>
-
-    Cache elements can also contain sub elements which take the same format of a factory class
-    and properties. Defined sub-elements are:
-
-    * cacheEventListenerFactory - Enables registration of listeners for cache events, such as
-      put, remove, update, and expire.
-
-    * bootstrapCacheLoaderFactory - Specifies a BootstrapCacheLoader, which is called by a
-      cache on initialisation to prepopulate itself.
-
-    * cacheExtensionFactory - Specifies a CacheExtension, a generic mechanism to tie a class
-      which holds a reference to a cache to the cache lifecycle.
-
-    * cacheExceptionHandlerFactory - Specifies a CacheExceptionHandler, which is called when
-      cache exceptions occur.
-
-    * cacheLoaderFactory - Specifies a CacheLoader, which can be used both asynchronously and
-      synchronously to load objects into a cache. More than one cacheLoaderFactory element
-      can be added, in which case the loaders form a chain which are executed in order. If a
-      loader returns null, the next in chain is called.
-
-    * copyStrategy - Specifies a fully qualified class which implements
-      net.sf.ehcache.store.compound.CopyStrategy. This strategy will be used for copyOnRead
-      and copyOnWrite in place of the default which is serialization.
-
-    Example of cache level resource tuning:
-    <cache name="memBound" maxBytesLocalHeap="100m" maxBytesLocalOffHeap="4g" maxBytesLocalDisk="200g" />
-
-
-    Cache Event Listeners
-    +++++++++++++++++++++
-
-    All cacheEventListenerFactory elements can take an optional property listenFor that describes
-    which events will be delivered in a clustered environment.  The listenFor attribute has the
-    following allowed values:
-
-    * all - the default is to deliver all local and remote events
-    * local - deliver only events originating in the current node
-    * remote - deliver only events originating in other nodes
-
-    Example of setting up a logging listener for local cache events:
-
-    <cacheEventListenerFactory class="my.company.log.CacheLogger"
-        listenFor="local" />
-
-
-    Search
-    ++++++
-
-    A <cache> can be made searchable by adding a <searchable/> sub-element. By default the keys
-    and value objects of elements put into the cache will be attributes against which
-    queries can be expressed.
-
-    <cache>
-        <searchable/>
-    </cache>
-
-
-    An "attribute" of the cache elements can also be defined to be searchable. In the example below
-    an attribute with the name "age" will be available for use in queries. The value for the "age"
-    attribute will be computed by calling the method "getAge()" on the value object of each element
-    in the cache. See net.sf.ehcache.search.attribute.ReflectionAttributeExtractor for the format of
-    attribute expressions. Attribute values must also conform to the set of types documented in the
-    net.sf.ehcache.search.attribute.AttributeExtractor interface
-
-    <cache>
-        <searchable>
-            <searchAttribute name="age" expression="value.getAge()"/>
-        </searchable>
-    </cache>
-
-
-    Attributes may also be defined using a JavaBean style. With the following attribute declaration
-    a public method getAge() will be expected to be found on either the key or value for cache elements
-
-    <cache>
-        <searchable>
-            <searchAttribute name="age"/>
-        </searchable>
-    </cache>
-
-    In more complex situations you can create your own attribute extractor by implementing the
-    AttributeExtractor interface. Providing your extractor class is shown in the following example:
-
-    <cache>
-        <searchable>
-            <searchAttribute name="age" class="com.example.MyAttributeExtractor"/>
-        </searchable>
-    </cache>
-
-    Use properties to pass state to your attribute extractor if needed. Your implementation must provide
-    a public constructor that takes a single java.util.Properties instance
-
-    <cache>
-        <searchable>
-            <searchAttribute name="age" class="com.example.MyAttributeExtractor" properties="foo=1,bar=2"/>
-        </searchable>
-    </cache>
-
-
-    RMI Cache Replication
-    +++++++++++++++++++++
-
-    Each cache that will be distributed needs to set a cache event listener which replicates
-    messages to the other CacheManager peers. For the built-in RMI implementation this is done
-    by adding a cacheEventListenerFactory element of type RMICacheReplicatorFactory to each
-    distributed cache's configuration as per the following example:
-
-    <cacheEventListenerFactory class="net.sf.ehcache.distribution.RMICacheReplicatorFactory"
-         properties="replicateAsynchronously=true,
-         replicatePuts=true,
-         replicatePutsViaCopy=false,
-         replicateUpdates=true,
-         replicateUpdatesViaCopy=true,
-         replicateRemovals=true,
-         asynchronousReplicationIntervalMillis=<number of milliseconds>,
-         asynchronousReplicationMaximumBatchSize=<number of operations>"
-         propertySeparator="," />
-
-    The RMICacheReplicatorFactory recognises the following properties:
-
-    * replicatePuts=true|false - whether new elements placed in a cache are
-      replicated to others. Defaults to true.
-
-    * replicatePutsViaCopy=true|false - whether the new elements are
-      copied to other caches (true), or whether a remove message is sent. Defaults to true.
-
-    * replicateUpdates=true|false - whether new elements which override an
-      element already existing with the same key are replicated. Defaults to true.
-
-    * replicateRemovals=true - whether element removals are replicated. Defaults to true.
-
-    * replicateAsynchronously=true | false - whether replications are
-      asynchronous (true) or synchronous (false). Defaults to true.
-
-    * replicateUpdatesViaCopy=true | false - whether the new elements are
-      copied to other caches (true), or whether a remove message is sent. Defaults to true.
-
-    * asynchronousReplicationIntervalMillis=<number of milliseconds> - The asynchronous
-      replicator runs at a set interval of milliseconds. The default is 1000. The minimum
-      is 10. This property is only applicable if replicateAsynchronously=true
-
-    * asynchronousReplicationMaximumBatchSize=<number of operations> - The maximum
-      number of operations that will be batch within a single RMI message.  The default
-      is 1000. This property is only applicable if replicateAsynchronously=true
-
-    JGroups Replication
-    +++++++++++++++++++
-
-    For the Jgroups replication this is done with:
-    <cacheEventListenerFactory class="net.sf.ehcache.distribution.jgroups.JGroupsCacheReplicatorFactory"
-                            properties="replicateAsynchronously=true, replicatePuts=true,
-               replicateUpdates=true, replicateUpdatesViaCopy=false,
-               replicateRemovals=true,asynchronousReplicationIntervalMillis=1000"/>
-    This listener supports the same properties as the RMICacheReplicationFactory.
-
-
-    JMS Replication
-    +++++++++++++++
-
-    For JMS-based replication this is done with:
-    <cacheEventListenerFactory
-          class="net.sf.ehcache.distribution.jms.JMSCacheReplicatorFactory"
-          properties="replicateAsynchronously=true,
-                       replicatePuts=true,
-                       replicateUpdates=true,
-                       replicateUpdatesViaCopy=true,
-                       replicateRemovals=true,
-                       asynchronousReplicationIntervalMillis=1000"
-           propertySeparator=","/>
-
-    This listener supports the same properties as the RMICacheReplicationFactory.
-
-    Cluster Bootstrapping
-    +++++++++++++++++++++
-
-    Bootstrapping a cluster may use a different mechanism to replication. e.g you can mix
-    JMS replication with bootstrap via RMI - just make sure you have the cacheManagerPeerProviderFactory
-    and cacheManagerPeerListenerFactory configured.
-
-    There are two bootstrapping mechanisms: RMI and JGroups.
-
-    RMI Bootstrap
-
-    The RMIBootstrapCacheLoader bootstraps caches in clusters where RMICacheReplicators are
-    used. It is configured as per the following example:
-
-    <bootstrapCacheLoaderFactory
-        class="net.sf.ehcache.distribution.RMIBootstrapCacheLoaderFactory"
-        properties="bootstrapAsynchronously=true, maximumChunkSizeBytes=5000000"
-        propertySeparator="," />
-
-    The RMIBootstrapCacheLoaderFactory recognises the following optional properties:
-
-    * bootstrapAsynchronously=true|false - whether the bootstrap happens in the background
-      after the cache has started. If false, bootstrapping must complete before the cache is
-      made available. The default value is true.
-
-    * maximumChunkSizeBytes=<integer> - Caches can potentially be very large, larger than the
-      memory limits of the VM. This property allows the bootstraper to fetched elements in
-      chunks. The default chunk size is 5000000 (5MB).
-
-    JGroups Bootstrap
-
-    Here is an example of bootstrap configuration using JGroups boostrap:
-
-    <bootstrapCacheLoaderFactory class="net.sf.ehcache.distribution.jgroups.JGroupsBootstrapCacheLoaderFactory"
-                                    properties="bootstrapAsynchronously=true"/>
-
-    The configuration properties are the same as for RMI above. Note that JGroups bootstrap only supports
-    asynchronous bootstrap mode.
-
-
-    Cache Exception Handling
-    ++++++++++++++++++++++++
-
-    By default, most cache operations will propagate a runtime CacheException on failure. An
-    interceptor, using a dynamic proxy, may be configured so that a CacheExceptionHandler can
-    be configured to intercept Exceptions. Errors are not intercepted.
-
-    It is configured as per the following example:
-
-      <cacheExceptionHandlerFactory class="com.example.ExampleExceptionHandlerFactory"
-                                      properties="logLevel=FINE"/>
-
-    Caches with ExceptionHandling configured are not of type Cache, but are of type Ehcache only,
-    and are not available using CacheManager.getCache(), but using CacheManager.getEhcache().
-
-
-    Cache Loader
-    ++++++++++++
-
-    A default CacheLoader may be set which loads objects into the cache through asynchronous and
-    synchronous methods on Cache. This is different to the bootstrap cache loader, which is used
-    only in distributed caching.
-
-    It is configured as per the following example:
-
-        <cacheLoaderFactory class="com.example.ExampleCacheLoaderFactory"
-                                      properties="type=int,startCounter=10"/>
-
-    Element value comparator
-    ++++++++++++++++++++++++
-
-    These two cache atomic methods:
-      removeElement(Element e)
-      replace(Element old, Element element)
-
-    rely on comparison of cached elements value. The default implementation relies on Object.equals()
-    but that can be changed in case you want to use a different way to compute equality of two elements.
-
-    This is configured as per the following example:
-
-    <elementValueComparator class="com.company.xyz.MyElementComparator"/>
-
-    The MyElementComparator class must implement the is net.sf.ehcache.store.ElementValueComparator
-    interface. The default implementation is net.sf.ehcache.store.DefaultElementValueComparator.
-
-
-    SizeOf Policy
-    +++++++++++++
-
-    Control how deep the SizeOf engine can go when sizing on-heap elements.
-
-    This is configured as per the following example:
-
-    <sizeOfPolicy maxDepth="100" maxDepthExceededBehavior="abort"/>
-
-    maxDepth controls how many linked objects can be visited before the SizeOf engine takes any action.
-    maxDepthExceededBehavior specifies what happens when the max depth is exceeded while sizing an object graph.
-     "continue" makes the SizeOf engine log a warning and continue the sizing. This is the default.
-     "abort"    makes the SizeOf engine abort the sizing, log a warning and mark the cache as not correctly tracking
-                memory usage. This makes Ehcache.hasAbortedSizeOf() return true when this happens.
-
-    The SizeOf policy can be configured at the cache manager level (directly under <ehcache>) and at
-    the cache level (under <cache> or <defaultCache>). The cache policy always overrides the cache manager
-    one if both are set. This element has no effect on distributed caches.
-
-    Transactions
-    ++++++++++++
-
-    To enable an ehcache as transactions, set the transactionalMode
-
-    transactionalMode="xa" - high performance JTA/XA implementation
-    transactionalMode="xa_strict" - canonically correct JTA/XA implementation
-    transactionMode="local" - high performance local transactions involving caches only
-    transactionalMode="off" - the default, no transactions
-
-    If set, all cache operations will need to be done through transactions.
-
-    To prevent users keeping references on stored elements and modifying them outside of any transaction's control,
-    transactions also require the cache to be configured copyOnRead and copyOnWrite.
-
-    CacheWriter
-    ++++++++++++
-
-    A CacheWriter can be set to write to an underlying resource. Only one CacheWriter can be
-    configured per cache.
-
-    The following is an example of how to configure CacheWriter for write-through:
-
-        <cacheWriter writeMode="write-through" notifyListenersOnException="true">
-            <cacheWriterFactory class="net.sf.ehcache.writer.TestCacheWriterFactory"
-                                properties="type=int,startCounter=10"/>
-        </cacheWriter>
-
-    The following is an example of how to configure CacheWriter for write-behind:
-
-        <cacheWriter writeMode="write-behind" minWriteDelay="1" maxWriteDelay="5"
-                     rateLimitPerSecond="5" writeCoalescing="true" writeBatching="true" writeBatchSize="1"
-                     retryAttempts="2" retryAttemptDelaySeconds="1">
-            <cacheWriterFactory class="net.sf.ehcache.writer.TestCacheWriterFactory"
-                                properties="type=int,startCounter=10"/>
-        </cacheWriter>
-
-    The cacheWriter element has the following attributes:
-    * writeMode: the write mode, write-through or write-behind
-
-    These attributes only apply to write-through mode:
-    * notifyListenersOnException: Sets whether to notify listeners when an exception occurs on a writer operation.
-
-    These attributes only apply to write-behind mode:
-    * minWriteDelay: Set the minimum number of seconds to wait before writing behind. If set to a value greater than 0,
-      it permits operations to build up in the queue. This is different from the maximum write delay in that by waiting
-      a minimum amount of time, work is always being built up. If the minimum write delay is set to zero and the
-      CacheWriter performs its work very quickly, the overhead of processing the write behind queue items becomes very
-      noticeable in a cluster since all the operations might be done for individual items instead of for a collection
-      of them.
-    * maxWriteDelay: Set the maximum number of seconds to wait before writing behind. If set to a value greater than 0,
-      it permits operations to build up in the queue to enable effective coalescing and batching optimisations.
-    * writeBatching: Sets whether to batch write operations. If set to true, writeAll and deleteAll will be called on
-      the CacheWriter rather than write and delete being called for each key. Resources such as databases can perform
-      more efficiently if updates are batched, thus reducing load.
-    * writeBatchSize: Sets the number of operations to include in each batch when writeBatching is enabled. If there are
-      less entries in the write-behind queue than the batch size, the queue length size is used.
-    * rateLimitPerSecond: Sets the maximum number of write operations to allow per second when writeBatching is enabled.
-    * writeCoalescing: Sets whether to use write coalescing. If set to true and multiple operations on the same key are
-      present in the write-behind queue, only the latest write is done, as the others are redundant.
-    * retryAttempts: Sets the number of times the operation is retried in the CacheWriter, this happens after the
-      original operation.
-    * retryAttemptDelaySeconds: Sets the number of seconds to wait before retrying an failed operation.
-
-    Cache Extension
-    +++++++++++++++
-
-    CacheExtensions are a general purpose mechanism to allow generic extensions to a Cache.
-    CacheExtensions are tied into the Cache lifecycle.
-
-    CacheExtensions are created using the CacheExtensionFactory which has a
-    <code>createCacheCacheExtension()</code> method which takes as a parameter a
-    Cache and properties. It can thus call back into any public method on Cache, including, of
-    course, the load methods.
-
-    Extensions are added as per the following example:
-
-         <cacheExtensionFactory class="com.example.FileWatchingCacheRefresherExtensionFactory"
-                             properties="refreshIntervalMillis=18000, loaderTimeout=3000,
-                                         flushPeriod=whatever, someOtherProperty=someValue ..."/>
-
-    Cache Decorator Factory
-    +++++++++++++++++++++++
-
-    Cache decorators can be configured directly in ehcache.xml. The decorators will be created and added to the CacheManager.
-    It accepts the name of a concrete class that extends net.sf.ehcache.constructs.CacheDecoratorFactory
-    The properties will be parsed according to the delimiter (default is comma ',') and passed to the concrete factory's
-    <code>createDecoratedEhcache(Ehcache cache, Properties properties)</code> method along with the reference to the owning cache.
-
-    It is configured as per the following example:
-
-        <cacheDecoratorFactory
-      class="com.company.DecoratedCacheFactory"
-      properties="property1=true ..." />
-
-    Distributed Caching with Terracotta
-    +++++++++++++++++++++++++++++++++++
-
-    Distributed Caches connect to a Terracotta Server Array. They are configured with the <terracotta> sub-element.
-
-    The <terracotta> sub-element has the following attributes:
-
-    * clustered=true|false - indicates whether this cache should be clustered (distributed) with Terracotta. By
-      default, if the <terracotta> element is included, clustered=true.
-
-    * valueMode=serialization|identity - the default is serialization
-
-      Indicates whether cache Elements are distributed with serialized copies or whether a single copy
-      in identity mode is distributed.
-
-      The implications of Identity mode should be clearly understood with reference to the Terracotta
-      documentation before use.
-
-    * copyOnRead=true|false - indicates whether cache values are deserialized on every read or if the
-      materialized cache value can be re-used between get() calls. This setting is useful if a cache
-      is being shared by callers with disparate classloaders or to prevent local drift if keys/values
-      are mutated locally without being put back in the cache.
-
-      The default is false.
-
-      Note: This setting is only relevant for caches with valueMode=serialization
-
-    * consistency=strong|eventual - Indicates whether this cache should have strong consistency or eventual
-      consistency. The default is eventual. See the documentation for the meaning of these terms.
-
-    * synchronousWrites=true|false
-
-      Synchronous writes (synchronousWrites="true")  maximize data safety by blocking the client thread until
-      the write has been written to the Terracotta Server Array.
-
-      This option is only available with consistency=strong. The default is false.
-
-    * concurrency - the number of segments that will be used by the map underneath the Terracotta Store.
-      Its optional and has default value of 0, which means will use default values based on the internal
-      Map being used underneath the store.
-
-      This value cannot be changed programmatically once a cache is initialized.
-
-    The <terracotta> sub-element also has a <nonstop> sub-element to allow configuration of cache behaviour if a distributed
-    cache operation cannot be completed within a set time or in the event of a clusterOffline message. If this element does not appear, nonstop behavior is off.
-
-    <nonstop> has the following attributes:
-
-    *  enabled="true" - defaults to true.
-
-    *  timeoutMillis - An SLA setting, so that if a cache operation takes longer than the allowed ms, it will timeout.
-
-    *  immediateTimeout="true|false" - What to do on receipt of a ClusterOffline event indicating that communications
-       with the Terracotta Server Array were interrupted.
-
-    <nonstop> has one sub-element, <timeoutBehavior> which has the following attribute:
-
-    *  type="noop|exception|localReads" - What to do when a timeout has occurred. Exception is the default.
-
-    Simplest example to indicate clustering:
-        <terracotta/>
-
-    To indicate the cache should not be clustered (or remove the <terracotta> element altogether):
-        <terracotta clustered="false"/>
-
-    To indicate the cache should be clustered using identity mode:
-        <terracotta clustered="true" valueMode="identity"/>
-
-    To indicate the cache should be clustered using "eventual" consistency mode for better performance :
-        <terracotta clustered="true" consistency="eventual"/>
-
-    To indicate the cache should be clustered using synchronous-write locking level:
-        <terracotta clustered="true" synchronousWrites="true"/>
-    -->
-
-    <!--
-    Default Cache configuration. These settings will be applied to caches
-    created programmatically using CacheManager.add(String cacheName).
-    This element is optional, and using CacheManager.add(String cacheName) when
-    its not present will throw CacheException
-
-    The defaultCache has an implicit name "default" which is a reserved cache name.
-    -->
-    <defaultCache
-            maxEntriesLocalHeap="10000"
-            eternal="false"
-            timeToIdleSeconds="120"
-            timeToLiveSeconds="120"
-            diskSpoolBufferSizeMB="30"
-            maxEntriesLocalDisk="10000000"
-            diskExpiryThreadIntervalSeconds="120"
-            memoryStoreEvictionPolicy="LRU"
-            statistics="false">
-        <persistence strategy="localTempSwap"/>
-    </defaultCache>
-
-    <!--
-    Sample caches. Following are some example caches. Remove these before use.
-    -->
-
-     <cache name="ldcache"
-           maxEntriesLocalHeap="10000"
-           maxEntriesLocalDisk="1000"
-           eternal="false"
-           diskSpoolBufferSizeMB="20"
-           timeToIdleSeconds="300"
-           timeToLiveSeconds="600"
-           memoryStoreEvictionPolicy="LFU"
-           transactionalMode="off">
-        <persistence strategy="localTempSwap"/>
-    </cache>
-
-
-
-</ehcache>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/libraries/ldcache/ldcache-backend-mapdb/pom.xml
----------------------------------------------------------------------
diff --git a/libraries/ldcache/ldcache-backend-mapdb/pom.xml b/libraries/ldcache/ldcache-backend-mapdb/pom.xml
deleted file mode 100644
index 14b21f6..0000000
--- a/libraries/ldcache/ldcache-backend-mapdb/pom.xml
+++ /dev/null
@@ -1,39 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-   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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.marmotta</groupId>
-        <artifactId>marmotta-parent</artifactId>
-        <version>3.2.0-SNAPSHOT</version>
-        <relativePath>../../../parent</relativePath>
-    </parent>
-
-    <artifactId>ldcache-backend-mapdb</artifactId>
-    <name>LDCache Backend: MapDB</name>
-
-    <description>
-        Implementation of an LDCache backend that stores all data (cache entries and triples) in a MapDB
-        (formerly JDBM) persistent store.
-    </description>
-
-    
-</project>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/libraries/ldcache/pom.xml
----------------------------------------------------------------------
diff --git a/libraries/ldcache/pom.xml b/libraries/ldcache/pom.xml
index 54bd88b..e905521 100644
--- a/libraries/ldcache/pom.xml
+++ b/libraries/ldcache/pom.xml
@@ -57,7 +57,6 @@
         <module>ldcache-core</module>
         <module>ldcache-sail-kiwi</module>
         <module>ldcache-sail-generic</module>
-        <module>ldcache-backend-ehcache</module>
         <module>ldcache-backend-kiwi</module>
         <module>ldcache-backend-file</module>
     </modules>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 0de87b6..62cf99f 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -958,12 +958,6 @@
             </dependency>
 
             <dependency>
-                <groupId>net.sf.ehcache</groupId>
-                <artifactId>ehcache-core</artifactId>
-                <version>2.6.6</version>
-            </dependency>
-
-            <dependency>
                 <groupId>org.infinispan</groupId>
                 <artifactId>infinispan-core</artifactId>
                 <version>6.0.0.Final</version>

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
----------------------------------------------------------------------
diff --git a/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java b/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
index 3718afe..979c806 100644
--- a/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
+++ b/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
@@ -112,6 +112,7 @@ public class KiWiStoreProvider implements StoreProvider {
         configuration.setDatacenterId(configurationService.getIntConfiguration(DATACENTER_ID,0));
         configuration.setFulltextEnabled(configurationService.getBooleanConfiguration(FULLTEXT_ENABLED, true));
         configuration.setFulltextLanguages(configurationService.getListConfiguration(FULLTEXT_LANGUAGES, ImmutableList.of("en")));
+        configuration.setClustered(configurationService.getBooleanConfiguration("database.clustered", false));
 
         if("native".equalsIgnoreCase(configurationService.getStringConfiguration(SPARQL_STRATEGY))) {
             return new KiWiSparqlSail(new KiWiStore(configuration));

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties
----------------------------------------------------------------------
diff --git a/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties b/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties
index 9ff7f78..65c013d 100644
--- a/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties
+++ b/platform/backends/marmotta-backend-kiwi/src/main/resources/config-defaults.properties
@@ -68,3 +68,7 @@ database.postgres.url = jdbc:postgresql://localhost:5432/lmf?prepareThreshold=3
 database.mysql.driver = com.mysql.jdbc.Driver
 database.mysql.url = jdbc:mysql://localhost:3306/lmf?useUnicode=true&characterEncoding=UTF-8&rewriteBatchedStatements=true
 
+
+# Turn on cluster-specific configuration options (e.g. replicated and distributed caching, synchronization, ...)
+database.clustered = false
+

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties
----------------------------------------------------------------------
diff --git a/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties b/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties
index 49c53a2..c214f3f 100644
--- a/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties
+++ b/platform/backends/marmotta-backend-kiwi/src/main/resources/config-descriptions.properties
@@ -53,3 +53,6 @@ database.fulltext.enabled.type = java.lang.Boolean
 
 database.fulltext.languages.description = list of languages supported by fulltext search; a fulltext index will be created for each language (PostgreSQL only)
 database.fulltext.languages.type = java.util.List
+
+database.clustered.description = Turn on cluster-specific configuration options (e.g. replicated and distributed caching, synchronization, ...)
+database.clustered.type = java.lang.Boolean

http://git-wip-us.apache.org/repos/asf/marmotta/blob/af9a8002/platform/marmotta-core/pom.xml
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/pom.xml b/platform/marmotta-core/pom.xml
index d4581b6..d38d15e 100644
--- a/platform/marmotta-core/pom.xml
+++ b/platform/marmotta-core/pom.xml
@@ -338,8 +338,12 @@
         
         <!-- Persistence -->
         <dependency>
-            <groupId>net.sf.ehcache</groupId>
-            <artifactId>ehcache-core</artifactId>
+            <groupId>org.infinispan</groupId>
+            <artifactId>infinispan-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.infinispan</groupId>
+            <artifactId>infinispan-cdi</artifactId>
         </dependency>
         <dependency>
             <groupId>javax.validation</groupId>


[6/7] git commit: bug fixes for MARMOTTA-388

Posted by ss...@apache.org.
bug fixes for MARMOTTA-388


Project: http://git-wip-us.apache.org/repos/asf/marmotta/repo
Commit: http://git-wip-us.apache.org/repos/asf/marmotta/commit/01c33dc3
Tree: http://git-wip-us.apache.org/repos/asf/marmotta/tree/01c33dc3
Diff: http://git-wip-us.apache.org/repos/asf/marmotta/diff/01c33dc3

Branch: refs/heads/develop
Commit: 01c33dc33d9c058634c3b80a111106e8cffca1fe
Parents: 4ba0d1b
Author: Sebastian Schaffert <ss...@apache.org>
Authored: Tue Dec 17 15:45:39 2013 +0100
Committer: Sebastian Schaffert <ss...@apache.org>
Committed: Tue Dec 17 15:45:39 2013 +0100

----------------------------------------------------------------------
 .../src/main/resources/ehcache-marmotta.xml     | 407 -------------------
 .../src/main/resources/ehcache-marmotta.xml     | 407 -------------------
 .../marmotta/kiwi/caching/KiWiCacheManager.java |  10 +
 .../backend/kiwi/KiWiStoreProvider.java         |   2 +-
 .../core/services/cache/CachingServiceImpl.java |  10 +-
 .../src/main/resources/jgroups-marmotta.xml     |  19 +-
 .../src/test/resources/ehcache-marmotta.xml     | 405 ------------------
 7 files changed, 38 insertions(+), 1222 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/marmotta/blob/01c33dc3/launchers/marmotta-webapp-bigdata/src/main/resources/ehcache-marmotta.xml
----------------------------------------------------------------------
diff --git a/launchers/marmotta-webapp-bigdata/src/main/resources/ehcache-marmotta.xml b/launchers/marmotta-webapp-bigdata/src/main/resources/ehcache-marmotta.xml
deleted file mode 100644
index eb01074..0000000
--- a/launchers/marmotta-webapp-bigdata/src/main/resources/ehcache-marmotta.xml
+++ /dev/null
@@ -1,407 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-
-    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.
-
--->
-<!--
-CacheManager Configuration
-==========================
-An ehcache-marmotta.xml corresponds to a single CacheManager.
-
-See instructions below or the ehcache schema (ehcache.xsd) on how to configure.
-
-System property tokens can be specified in this file which are replaced when the configuration
-is loaded. For example multicastGroupPort=${multicastGroupPort} can be replaced with the
-System property either from an environment variable or a system property specified with a
-command line switch such as -DmulticastGroupPort=4446.
-
-The attributes of <ehcache> are:
-* name - an optional name for the CacheManager.  The name is optional and primarily used 
-for documentation or to distinguish Terracotta clustered cache state.  With Terracotta 
-clustered caches, a combination of CacheManager name and cache name uniquely identify a 
-particular cache store in the Terracotta clustered memory.
-* updateCheck - an optional boolean flag specifying whether this CacheManager should check
-for new versions of Ehcache over the Internet.  If not specified, updateCheck="true".
-* monitoring - an optional setting that determines whether the CacheManager should 
-automatically register the SampledCacheMBean with the system MBean server.  Currently,
-this monitoring is only useful when using Terracotta and thus the "autodetect" value 
-will detect the presence of Terracotta and register the MBean.  Other allowed values 
-are "on" and "off".  The default is "autodetect".
--->    
-<ehcache xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:noNamespaceSchemaLocation="http://ehcache.org/ehcache.xsd"
-         updateCheck="false"
-         name="lmf">
-
-    <!-- 
-    DiskStore configuration
-    =======================
-
-    The diskStore element is optional. To turn off disk store path creation, comment out the diskStore
-    element below.
-
-    Configure it if you have overflowToDisk or diskPersistent enabled for any cache.
-
-    If it is not configured, and a cache is created which requires a disk store, a warning will be
-     issued and java.io.tmpdir will automatically be used.
-
-    diskStore has only one attribute - "path". It is the path to the directory where
-    .data and .index files will be created.
-
-    If the path is one of the following Java System Property it is replaced by its value in the
-    running VM. For backward compatibility these are not specified without being enclosed in the ${token}
-    replacement syntax.
-
-    The following properties are translated:
-    * user.home - KiWiUser's home directory
-    * user.dir - KiWiUser's current working directory
-    * java.io.tmpdir - Default temp file path
-    * ehcache.disk.store.dir - A system property you would normally specify on the command line
-      e.g. java -Dehcache.disk.store.dir=/u01/myapp/diskdir ...
-
-    Subdirectories can be specified below the property e.g. java.io.tmpdir/one
-
-    -->
-    <!--
-    <diskStore path="java.io.tmpdir/lmf-cache/lmf"/>
-    -->
-   <!--
-    Cachemanagereventlistener
-    =========================
-    Specifies a CacheManagerEventListenerFactory which is notified when Caches are added
-    or removed from the CacheManager.
-
-    The attributes of CacheManagerEventListenerFactory are:
-    * class - a fully qualified factory class name
-    * properties - comma separated properties having meaning only to the factory.
-
-    Sets the fully qualified class name to be registered as the CacheManager event listener.
-
-    The events include:
-    * adding a Cache
-    * removing a Cache
-
-    Callbacks to listener methods are synchronous and unsynchronized. It is the responsibility
-    of the implementer to safely handle the potential performance and thread safety issues
-    depending on what their listener is doing.
-
-    If no class is specified, no listener is created. There is no default.
-    -->
-    <cacheManagerEventListenerFactory class="" properties=""/>
-
-
-    <!--
-    CacheManagerPeerProvider
-    ========================
-    (For distributed operation)
-
-    Specifies a CacheManagerPeerProviderFactory which will be used to create a
-    CacheManagerPeerProvider, which discovers other CacheManagers in the cluster.
-
-    One or more providers can be configured. The first one in the ehcache-marmotta.xml is the default, which is used
-    for replication and bootstrapping.
-
-    The attributes of cacheManagerPeerProviderFactory are:
-    * class - a fully qualified factory class name
-    * properties - comma separated properties having meaning only to the factory.
-
-    Providers are available for RMI, JGroups and JMS as shown following.
-
-    RMICacheManagerPeerProvider
-    +++++++++++++++++++++++++++
-
-    Ehcache comes with a built-in RMI-based distribution system with two means of discovery of
-    CacheManager peers participating in the cluster:
-    * automatic, using a multicast group. This one automatically discovers peers and detects
-      changes such as peers entering and leaving the group
-    * manual, using manual rmiURL configuration. A hardcoded list of peers is provided at
-      configuration time.
-
-    Configuring Automatic Discovery:
-    Automatic discovery is configured as per the following example:
-    <cacheManagerPeerProviderFactory
-                        class="net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                        properties="hostName=fully_qualified_hostname_or_ip,
-                                    peerDiscovery=automatic, multicastGroupAddress=230.0.0.1,
-                                    multicastGroupPort=4446, timeToLive=32"/>
-
-    Valid properties are:
-    * peerDiscovery (mandatory) - specify "automatic"
-    * multicastGroupAddress (mandatory) - specify a valid multicast group address
-    * multicastGroupPort (mandatory) - specify a dedicated port for the multicast heartbeat
-      traffic
-    * timeToLive - specify a value between 0 and 255 which determines how far the packets will
-      propagate.
-
-      By convention, the restrictions are:
-      0   - the same host
-      1   - the same subnet
-      32  - the same site
-      64  - the same region
-      128 - the same continent
-      255 - unrestricted
-
-     * hostName - the hostname or IP of the interface to be used for sending and receiving multicast packets
-       (relevant to mulithomed hosts only)
-
-    Configuring Manual Discovery:
-    Manual discovery requires a unique configuration per host. It is contains a list of rmiURLs for the peers, other
-    than itself. So, if we have server1, server2 and server3 the configuration will be:
-
-    In server1's configuration:
-    <cacheManagerPeerProviderFactory class=
-                          "net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                          properties="peerDiscovery=manual,
-                          rmiUrls=//server2:40000/sampleCache1|//server3:40000/sampleCache1
-                          | //server2:40000/sampleCache2|//server3:40000/sampleCache2"
-                          propertySeparator="," />
-
-    In server2's configuration:
-    <cacheManagerPeerProviderFactory class=
-                          "net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                          properties="peerDiscovery=manual,
-                          rmiUrls=//server1:40000/sampleCache1|//server3:40000/sampleCache1
-                          | //server1:40000/sampleCache2|//server3:40000/sampleCache2"
-                          propertySeparator="," />
-
-    In server3's configuration:
-    <cacheManagerPeerProviderFactory class=
-                          "net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                          properties="peerDiscovery=manual,
-                          rmiUrls=//server1:40000/sampleCache1|//server2:40000/sampleCache1
-                          | //server1:40000/sampleCache2|//server2:40000/sampleCache2"
-                          propertySeparator="," />
-
-
-    Valid properties are:
-    * peerDiscovery (mandatory) - specify "manual"
-    * rmiUrls (mandatory) - specify a pipe separated list of rmiUrls, in the form
-                            //hostname:port
-    * hostname (optional) - the hostname is the hostname of the remote CacheManager peer. The port is the listening
-      port of the RMICacheManagerPeerListener of the remote CacheManager peer.
-    
-    JGroupsCacheManagerPeerProvider
-    +++++++++++++++++++++++++++++++
-    <cacheManagerPeerProviderFactory class="net.sf.ehcache.distribution.jgroups.JGroupsCacheManagerPeerProviderFactory"
-                                     properties="connect=UDP(mcast_addr=231.12.21.132;mcast_port=45566;ip_ttl=32;
-                                     mcast_send_buf_size=150000;mcast_recv_buf_size=80000):
-                                     PING(timeout=2000;num_initial_members=6):
-                                     MERGE2(min_interval=5000;max_interval=10000):
-                                     FD_SOCK:VERIFY_SUSPECT(timeout=1500):
-                                     pbcast.NAKACK(gc_lag=10;retransmit_timeout=3000):
-                                     UNICAST(timeout=5000):
-                                     pbcast.STABLE(desired_avg_gossip=20000):
-                                     FRAG:
-                                     pbcast.GMS(join_timeout=5000;join_retry_timeout=2000;shun=false;print_local_addr=false)"
-                                     propertySeparator="::"
-            />
-     The only property necessary is the connect String used by jgroups to configure itself. Refer to the Jgroups documentation for explanation
-     of all the protocols. The example above uses UDP multicast. If the connect property is not specified the default JGroups connection will be
-     used.       
-
-
-    JMSCacheManagerPeerProviderFactory
-    ++++++++++++++++++++++++++++++++++
-    <cacheManagerPeerProviderFactory
-            class="net.sf.ehcache.distribution.jms.JMSCacheManagerPeerProviderFactory"
-            properties="..."
-            propertySeparator=","
-            />
-
-    The JMS PeerProviderFactory uses JNDI to maintain message queue independence. Refer to the manual for full configuration
-    examples using ActiveMQ and Open Message Queue.
-
-    Valid properties are:
-    * initialContextFactoryName (mandatory) - the name of the factory used to create the message queue initial context.
-    * providerURL (mandatory) - the JNDI configuration information for the service provider to use.
-    * topicConnectionFactoryBindingName (mandatory) - the JNDI binding name for the TopicConnectionFactory
-    * topicBindingName (mandatory) - the JNDI binding name for the topic name
-    * getQueueBindingName (mandatory only if using jmsCacheLoader) - the JNDI binding name for the queue name
-    * securityPrincipalName - the JNDI java.naming.security.principal
-    * securityCredentials - the JNDI java.naming.security.credentials
-    * urlPkgPrefixes - the JNDI java.naming.factory.url.pkgs
-    * userName - the user name to use when creating the TopicConnection to the Message Queue
-    * password - the password to use when creating the TopicConnection to the Message Queue
-    * acknowledgementMode - the JMS Acknowledgement mode for both publisher and subscriber. The available choices are
-                            AUTO_ACKNOWLEDGE, DUPS_OK_ACKNOWLEDGE and SESSION_TRANSACTED. The default is AUTO_ACKNOWLEDGE.
-    -->
-<!--    <cacheManagerPeerProviderFactory-->
-<!--            class="net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"-->
-<!--            properties="peerDiscovery=automatic,-->
-<!--                        multicastGroupAddress=230.0.0.1,-->
-<!--                        multicastGroupPort=4446, timeToLive=1"-->
-<!--            propertySeparator=","-->
-<!--            />-->
-
-
-    <!--
-    CacheManagerPeerListener
-    ========================
-    (Enable for distributed operation)
-
-    Specifies a CacheManagerPeerListenerFactory which will be used to create a
-    CacheManagerPeerListener, which listens for messages from cache replicators participating in the cluster.
-
-    The attributes of cacheManagerPeerListenerFactory are:
-    class - a fully qualified factory class name
-    properties - comma separated properties having meaning only to the factory.
-
-    Ehcache comes with a built-in RMI-based distribution system. The listener component is
-    RMICacheManagerPeerListener which is configured using
-    RMICacheManagerPeerListenerFactory. It is configured as per the following example:
-
-    <cacheManagerPeerListenerFactory
-        class="net.sf.ehcache.distribution.RMICacheManagerPeerListenerFactory"
-        properties="hostName=fully_qualified_hostname_or_ip,
-                    port=40001,
-                    remoteObjectPort=40002,
-                    socketTimeoutMillis=120000"
-                    propertySeparator="," />
-
-    All properties are optional. They are:
-    * hostName - the hostName of the host the listener is running on. Specify
-      where the host is multihomed and you want to control the interface over which cluster
-      messages are received. Defaults to the host name of the default interface if not
-      specified.
-    * port - the port the RMI Registry listener listens on. This defaults to a free port if not specified.
-    * remoteObjectPort - the port number on which the remote objects bound in the registry receive calls.
-                         This defaults to a free port if not specified.
-    * socketTimeoutMillis - the number of ms client sockets will stay open when sending
-      messages to the listener. This should be long enough for the slowest message.
-      If not specified it defaults to 120000ms.
-
-    -->
-<!--    <cacheManagerPeerListenerFactory-->
-<!--            class="net.sf.ehcache.distribution.RMICacheManagerPeerListenerFactory"/>-->
-
-    <!-- Cache configuration.
-
-    The following attributes are required.
-
-    name:
-    Sets the name of the cache. This is used to identify the cache. It must be unique.
-
-    maxElementsInMemory:
-    Sets the maximum number of objects that will be created in memory
-
-        maxElementsOnDisk:
-    Sets the maximum number of objects that will be maintained in the DiskStore
-        The default value is zero, meaning unlimited.
-
-    eternal:
-    Sets whether elements are eternal. If eternal,  timeouts are ignored and the
-    element is never expired.
-
-    overflowToDisk:
-    Sets whether elements can overflow to disk when the memory store
-    has reached the maxInMemory limit.
-
-    The following attributes are optional.
-
-    timeToIdleSeconds:
-    Sets the time to idle for an element before it expires.
-    i.e. The maximum amount of time between accesses before an element expires
-    Is only used if the element is not eternal.
-    Optional attribute. A value of 0 means that an Element can idle for infinity.
-    The default value is 0.
-
-    timeToLiveSeconds:
-    Sets the time to live for an element before it expires.
-    i.e. The maximum time between creation time and when an element expires.
-    Is only used if the element is not eternal.
-    Optional attribute. A value of 0 means that and Element can live for infinity.
-    The default value is 0.
-
-    diskPersistent:
-    Whether the disk store persists between restarts of the Virtual Machine.
-    The default value is false.
-
-    diskExpiryThreadIntervalSeconds:
-    The number of seconds between runs of the disk expiry thread. The default value
-    is 120 seconds.
-
-    memoryStoreEvictionPolicy:
-    Policy would be enforced upon reaching the maxElementsInMemory limit. Default
-    policy is Least Recently Used (specified as LRU). Other policies available -
-    First In First Out (specified as FIFO) and Less Frequently Used
-    (specified as LFU)
-
-    -->
-
-    <!--
-    Mandatory Default Cache configuration. These settings will be applied to caches
-    created programmtically using CacheManager.add(String cacheName)
-    -->
-    <defaultCache
-            maxElementsInMemory="20000"
-            overflowToDisk="false"
-            memoryStoreEvictionPolicy="LRU"
-            />
-
-
-    <!-- the cache used for triple queries by KiWi -->
-    <cache name="uri-node-cache"
-           statistics="true"
-           maxElementsInMemory="100000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-    <cache name="anon-node-cache"
-           statistics="true"
-           maxElementsInMemory="10000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-    <cache name="literal-cache"
-           statistics="true"
-           maxElementsInMemory="10000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-
-    <cache name="namespace-prefix-cache"
-           statistics="true"
-           maxElementsInMemory="100"
-           overflowToDisk="false"/>
-
-    <cache name="namespace-uri-cache"
-           statistics="true"
-           maxElementsInMemory="100"
-           overflowToDisk="false"/>
-
-    <!-- the cache used for triple queries by KiWi -->
-    <cache name="query-cache"
-           statistics="true"
-           maxElementsInMemory="200000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-
-    <!-- the cache used for resource lookups from module jar files -->
-    <cache name="resource-cache"
-           statistics="true"
-           maxElementsInMemory="10000"
-           timeToIdleSeconds="3600"
-           timeToLiveSeconds="3600"
-           overflowToDisk="false"
-           memoryStoreEvictionPolicy="LRU"/>
-
-    <!--  uncomment to enable cache debugging -->
-<!-- 
-	<cacheManagerPeerListenerFactory
-	    class="org.terracotta.ehcachedx.monitor.probe.ProbePeerListenerFactory"
-	    properties="monitorAddress=localhost, monitorPort=9889" />
--->
-
-</ehcache>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/marmotta/blob/01c33dc3/launchers/marmotta-webapp/src/main/resources/ehcache-marmotta.xml
----------------------------------------------------------------------
diff --git a/launchers/marmotta-webapp/src/main/resources/ehcache-marmotta.xml b/launchers/marmotta-webapp/src/main/resources/ehcache-marmotta.xml
deleted file mode 100644
index eb01074..0000000
--- a/launchers/marmotta-webapp/src/main/resources/ehcache-marmotta.xml
+++ /dev/null
@@ -1,407 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-
-    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.
-
--->
-<!--
-CacheManager Configuration
-==========================
-An ehcache-marmotta.xml corresponds to a single CacheManager.
-
-See instructions below or the ehcache schema (ehcache.xsd) on how to configure.
-
-System property tokens can be specified in this file which are replaced when the configuration
-is loaded. For example multicastGroupPort=${multicastGroupPort} can be replaced with the
-System property either from an environment variable or a system property specified with a
-command line switch such as -DmulticastGroupPort=4446.
-
-The attributes of <ehcache> are:
-* name - an optional name for the CacheManager.  The name is optional and primarily used 
-for documentation or to distinguish Terracotta clustered cache state.  With Terracotta 
-clustered caches, a combination of CacheManager name and cache name uniquely identify a 
-particular cache store in the Terracotta clustered memory.
-* updateCheck - an optional boolean flag specifying whether this CacheManager should check
-for new versions of Ehcache over the Internet.  If not specified, updateCheck="true".
-* monitoring - an optional setting that determines whether the CacheManager should 
-automatically register the SampledCacheMBean with the system MBean server.  Currently,
-this monitoring is only useful when using Terracotta and thus the "autodetect" value 
-will detect the presence of Terracotta and register the MBean.  Other allowed values 
-are "on" and "off".  The default is "autodetect".
--->    
-<ehcache xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:noNamespaceSchemaLocation="http://ehcache.org/ehcache.xsd"
-         updateCheck="false"
-         name="lmf">
-
-    <!-- 
-    DiskStore configuration
-    =======================
-
-    The diskStore element is optional. To turn off disk store path creation, comment out the diskStore
-    element below.
-
-    Configure it if you have overflowToDisk or diskPersistent enabled for any cache.
-
-    If it is not configured, and a cache is created which requires a disk store, a warning will be
-     issued and java.io.tmpdir will automatically be used.
-
-    diskStore has only one attribute - "path". It is the path to the directory where
-    .data and .index files will be created.
-
-    If the path is one of the following Java System Property it is replaced by its value in the
-    running VM. For backward compatibility these are not specified without being enclosed in the ${token}
-    replacement syntax.
-
-    The following properties are translated:
-    * user.home - KiWiUser's home directory
-    * user.dir - KiWiUser's current working directory
-    * java.io.tmpdir - Default temp file path
-    * ehcache.disk.store.dir - A system property you would normally specify on the command line
-      e.g. java -Dehcache.disk.store.dir=/u01/myapp/diskdir ...
-
-    Subdirectories can be specified below the property e.g. java.io.tmpdir/one
-
-    -->
-    <!--
-    <diskStore path="java.io.tmpdir/lmf-cache/lmf"/>
-    -->
-   <!--
-    Cachemanagereventlistener
-    =========================
-    Specifies a CacheManagerEventListenerFactory which is notified when Caches are added
-    or removed from the CacheManager.
-
-    The attributes of CacheManagerEventListenerFactory are:
-    * class - a fully qualified factory class name
-    * properties - comma separated properties having meaning only to the factory.
-
-    Sets the fully qualified class name to be registered as the CacheManager event listener.
-
-    The events include:
-    * adding a Cache
-    * removing a Cache
-
-    Callbacks to listener methods are synchronous and unsynchronized. It is the responsibility
-    of the implementer to safely handle the potential performance and thread safety issues
-    depending on what their listener is doing.
-
-    If no class is specified, no listener is created. There is no default.
-    -->
-    <cacheManagerEventListenerFactory class="" properties=""/>
-
-
-    <!--
-    CacheManagerPeerProvider
-    ========================
-    (For distributed operation)
-
-    Specifies a CacheManagerPeerProviderFactory which will be used to create a
-    CacheManagerPeerProvider, which discovers other CacheManagers in the cluster.
-
-    One or more providers can be configured. The first one in the ehcache-marmotta.xml is the default, which is used
-    for replication and bootstrapping.
-
-    The attributes of cacheManagerPeerProviderFactory are:
-    * class - a fully qualified factory class name
-    * properties - comma separated properties having meaning only to the factory.
-
-    Providers are available for RMI, JGroups and JMS as shown following.
-
-    RMICacheManagerPeerProvider
-    +++++++++++++++++++++++++++
-
-    Ehcache comes with a built-in RMI-based distribution system with two means of discovery of
-    CacheManager peers participating in the cluster:
-    * automatic, using a multicast group. This one automatically discovers peers and detects
-      changes such as peers entering and leaving the group
-    * manual, using manual rmiURL configuration. A hardcoded list of peers is provided at
-      configuration time.
-
-    Configuring Automatic Discovery:
-    Automatic discovery is configured as per the following example:
-    <cacheManagerPeerProviderFactory
-                        class="net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                        properties="hostName=fully_qualified_hostname_or_ip,
-                                    peerDiscovery=automatic, multicastGroupAddress=230.0.0.1,
-                                    multicastGroupPort=4446, timeToLive=32"/>
-
-    Valid properties are:
-    * peerDiscovery (mandatory) - specify "automatic"
-    * multicastGroupAddress (mandatory) - specify a valid multicast group address
-    * multicastGroupPort (mandatory) - specify a dedicated port for the multicast heartbeat
-      traffic
-    * timeToLive - specify a value between 0 and 255 which determines how far the packets will
-      propagate.
-
-      By convention, the restrictions are:
-      0   - the same host
-      1   - the same subnet
-      32  - the same site
-      64  - the same region
-      128 - the same continent
-      255 - unrestricted
-
-     * hostName - the hostname or IP of the interface to be used for sending and receiving multicast packets
-       (relevant to mulithomed hosts only)
-
-    Configuring Manual Discovery:
-    Manual discovery requires a unique configuration per host. It is contains a list of rmiURLs for the peers, other
-    than itself. So, if we have server1, server2 and server3 the configuration will be:
-
-    In server1's configuration:
-    <cacheManagerPeerProviderFactory class=
-                          "net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                          properties="peerDiscovery=manual,
-                          rmiUrls=//server2:40000/sampleCache1|//server3:40000/sampleCache1
-                          | //server2:40000/sampleCache2|//server3:40000/sampleCache2"
-                          propertySeparator="," />
-
-    In server2's configuration:
-    <cacheManagerPeerProviderFactory class=
-                          "net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                          properties="peerDiscovery=manual,
-                          rmiUrls=//server1:40000/sampleCache1|//server3:40000/sampleCache1
-                          | //server1:40000/sampleCache2|//server3:40000/sampleCache2"
-                          propertySeparator="," />
-
-    In server3's configuration:
-    <cacheManagerPeerProviderFactory class=
-                          "net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                          properties="peerDiscovery=manual,
-                          rmiUrls=//server1:40000/sampleCache1|//server2:40000/sampleCache1
-                          | //server1:40000/sampleCache2|//server2:40000/sampleCache2"
-                          propertySeparator="," />
-
-
-    Valid properties are:
-    * peerDiscovery (mandatory) - specify "manual"
-    * rmiUrls (mandatory) - specify a pipe separated list of rmiUrls, in the form
-                            //hostname:port
-    * hostname (optional) - the hostname is the hostname of the remote CacheManager peer. The port is the listening
-      port of the RMICacheManagerPeerListener of the remote CacheManager peer.
-    
-    JGroupsCacheManagerPeerProvider
-    +++++++++++++++++++++++++++++++
-    <cacheManagerPeerProviderFactory class="net.sf.ehcache.distribution.jgroups.JGroupsCacheManagerPeerProviderFactory"
-                                     properties="connect=UDP(mcast_addr=231.12.21.132;mcast_port=45566;ip_ttl=32;
-                                     mcast_send_buf_size=150000;mcast_recv_buf_size=80000):
-                                     PING(timeout=2000;num_initial_members=6):
-                                     MERGE2(min_interval=5000;max_interval=10000):
-                                     FD_SOCK:VERIFY_SUSPECT(timeout=1500):
-                                     pbcast.NAKACK(gc_lag=10;retransmit_timeout=3000):
-                                     UNICAST(timeout=5000):
-                                     pbcast.STABLE(desired_avg_gossip=20000):
-                                     FRAG:
-                                     pbcast.GMS(join_timeout=5000;join_retry_timeout=2000;shun=false;print_local_addr=false)"
-                                     propertySeparator="::"
-            />
-     The only property necessary is the connect String used by jgroups to configure itself. Refer to the Jgroups documentation for explanation
-     of all the protocols. The example above uses UDP multicast. If the connect property is not specified the default JGroups connection will be
-     used.       
-
-
-    JMSCacheManagerPeerProviderFactory
-    ++++++++++++++++++++++++++++++++++
-    <cacheManagerPeerProviderFactory
-            class="net.sf.ehcache.distribution.jms.JMSCacheManagerPeerProviderFactory"
-            properties="..."
-            propertySeparator=","
-            />
-
-    The JMS PeerProviderFactory uses JNDI to maintain message queue independence. Refer to the manual for full configuration
-    examples using ActiveMQ and Open Message Queue.
-
-    Valid properties are:
-    * initialContextFactoryName (mandatory) - the name of the factory used to create the message queue initial context.
-    * providerURL (mandatory) - the JNDI configuration information for the service provider to use.
-    * topicConnectionFactoryBindingName (mandatory) - the JNDI binding name for the TopicConnectionFactory
-    * topicBindingName (mandatory) - the JNDI binding name for the topic name
-    * getQueueBindingName (mandatory only if using jmsCacheLoader) - the JNDI binding name for the queue name
-    * securityPrincipalName - the JNDI java.naming.security.principal
-    * securityCredentials - the JNDI java.naming.security.credentials
-    * urlPkgPrefixes - the JNDI java.naming.factory.url.pkgs
-    * userName - the user name to use when creating the TopicConnection to the Message Queue
-    * password - the password to use when creating the TopicConnection to the Message Queue
-    * acknowledgementMode - the JMS Acknowledgement mode for both publisher and subscriber. The available choices are
-                            AUTO_ACKNOWLEDGE, DUPS_OK_ACKNOWLEDGE and SESSION_TRANSACTED. The default is AUTO_ACKNOWLEDGE.
-    -->
-<!--    <cacheManagerPeerProviderFactory-->
-<!--            class="net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"-->
-<!--            properties="peerDiscovery=automatic,-->
-<!--                        multicastGroupAddress=230.0.0.1,-->
-<!--                        multicastGroupPort=4446, timeToLive=1"-->
-<!--            propertySeparator=","-->
-<!--            />-->
-
-
-    <!--
-    CacheManagerPeerListener
-    ========================
-    (Enable for distributed operation)
-
-    Specifies a CacheManagerPeerListenerFactory which will be used to create a
-    CacheManagerPeerListener, which listens for messages from cache replicators participating in the cluster.
-
-    The attributes of cacheManagerPeerListenerFactory are:
-    class - a fully qualified factory class name
-    properties - comma separated properties having meaning only to the factory.
-
-    Ehcache comes with a built-in RMI-based distribution system. The listener component is
-    RMICacheManagerPeerListener which is configured using
-    RMICacheManagerPeerListenerFactory. It is configured as per the following example:
-
-    <cacheManagerPeerListenerFactory
-        class="net.sf.ehcache.distribution.RMICacheManagerPeerListenerFactory"
-        properties="hostName=fully_qualified_hostname_or_ip,
-                    port=40001,
-                    remoteObjectPort=40002,
-                    socketTimeoutMillis=120000"
-                    propertySeparator="," />
-
-    All properties are optional. They are:
-    * hostName - the hostName of the host the listener is running on. Specify
-      where the host is multihomed and you want to control the interface over which cluster
-      messages are received. Defaults to the host name of the default interface if not
-      specified.
-    * port - the port the RMI Registry listener listens on. This defaults to a free port if not specified.
-    * remoteObjectPort - the port number on which the remote objects bound in the registry receive calls.
-                         This defaults to a free port if not specified.
-    * socketTimeoutMillis - the number of ms client sockets will stay open when sending
-      messages to the listener. This should be long enough for the slowest message.
-      If not specified it defaults to 120000ms.
-
-    -->
-<!--    <cacheManagerPeerListenerFactory-->
-<!--            class="net.sf.ehcache.distribution.RMICacheManagerPeerListenerFactory"/>-->
-
-    <!-- Cache configuration.
-
-    The following attributes are required.
-
-    name:
-    Sets the name of the cache. This is used to identify the cache. It must be unique.
-
-    maxElementsInMemory:
-    Sets the maximum number of objects that will be created in memory
-
-        maxElementsOnDisk:
-    Sets the maximum number of objects that will be maintained in the DiskStore
-        The default value is zero, meaning unlimited.
-
-    eternal:
-    Sets whether elements are eternal. If eternal,  timeouts are ignored and the
-    element is never expired.
-
-    overflowToDisk:
-    Sets whether elements can overflow to disk when the memory store
-    has reached the maxInMemory limit.
-
-    The following attributes are optional.
-
-    timeToIdleSeconds:
-    Sets the time to idle for an element before it expires.
-    i.e. The maximum amount of time between accesses before an element expires
-    Is only used if the element is not eternal.
-    Optional attribute. A value of 0 means that an Element can idle for infinity.
-    The default value is 0.
-
-    timeToLiveSeconds:
-    Sets the time to live for an element before it expires.
-    i.e. The maximum time between creation time and when an element expires.
-    Is only used if the element is not eternal.
-    Optional attribute. A value of 0 means that and Element can live for infinity.
-    The default value is 0.
-
-    diskPersistent:
-    Whether the disk store persists between restarts of the Virtual Machine.
-    The default value is false.
-
-    diskExpiryThreadIntervalSeconds:
-    The number of seconds between runs of the disk expiry thread. The default value
-    is 120 seconds.
-
-    memoryStoreEvictionPolicy:
-    Policy would be enforced upon reaching the maxElementsInMemory limit. Default
-    policy is Least Recently Used (specified as LRU). Other policies available -
-    First In First Out (specified as FIFO) and Less Frequently Used
-    (specified as LFU)
-
-    -->
-
-    <!--
-    Mandatory Default Cache configuration. These settings will be applied to caches
-    created programmtically using CacheManager.add(String cacheName)
-    -->
-    <defaultCache
-            maxElementsInMemory="20000"
-            overflowToDisk="false"
-            memoryStoreEvictionPolicy="LRU"
-            />
-
-
-    <!-- the cache used for triple queries by KiWi -->
-    <cache name="uri-node-cache"
-           statistics="true"
-           maxElementsInMemory="100000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-    <cache name="anon-node-cache"
-           statistics="true"
-           maxElementsInMemory="10000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-    <cache name="literal-cache"
-           statistics="true"
-           maxElementsInMemory="10000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-
-    <cache name="namespace-prefix-cache"
-           statistics="true"
-           maxElementsInMemory="100"
-           overflowToDisk="false"/>
-
-    <cache name="namespace-uri-cache"
-           statistics="true"
-           maxElementsInMemory="100"
-           overflowToDisk="false"/>
-
-    <!-- the cache used for triple queries by KiWi -->
-    <cache name="query-cache"
-           statistics="true"
-           maxElementsInMemory="200000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-
-    <!-- the cache used for resource lookups from module jar files -->
-    <cache name="resource-cache"
-           statistics="true"
-           maxElementsInMemory="10000"
-           timeToIdleSeconds="3600"
-           timeToLiveSeconds="3600"
-           overflowToDisk="false"
-           memoryStoreEvictionPolicy="LRU"/>
-
-    <!--  uncomment to enable cache debugging -->
-<!-- 
-	<cacheManagerPeerListenerFactory
-	    class="org.terracotta.ehcachedx.monitor.probe.ProbePeerListenerFactory"
-	    properties="monitorAddress=localhost, monitorPort=9889" />
--->
-
-</ehcache>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/marmotta/blob/01c33dc3/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
index 6c25504..b68d7a9 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
@@ -28,6 +28,9 @@ import org.infinispan.distribution.ch.SyncConsistentHashFactory;
 import org.infinispan.eviction.EvictionStrategy;
 import org.infinispan.manager.DefaultCacheManager;
 import org.infinispan.manager.EmbeddedCacheManager;
+import org.infinispan.remoting.transport.Address;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.Iterator;
 import java.util.Set;
@@ -40,6 +43,7 @@ import java.util.concurrent.TimeUnit;
  */
 public class KiWiCacheManager {
 
+    private static Logger log = LoggerFactory.getLogger(KiWiCacheManager.class);
 
     public static final String NODE_CACHE = "node-cache";
     public static final String TRIPLE_CACHE = "triple-cache";
@@ -114,6 +118,12 @@ public class KiWiCacheManager {
 
 
         cacheManager = new DefaultCacheManager(globalConfiguration, defaultConfiguration, true);
+        if(log.isInfoEnabled()) {
+            log.info("Members in Apache Marmotta KiWi cache cluster:");
+            for(Address a : cacheManager.getMembers()) {
+                log.info(" - {}",a);
+            }
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/marmotta/blob/01c33dc3/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
----------------------------------------------------------------------
diff --git a/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java b/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
index b7d63e2..8609c9d 100644
--- a/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
+++ b/platform/backends/marmotta-backend-kiwi/src/main/java/org/apache/marmotta/platform/backend/kiwi/KiWiStoreProvider.java
@@ -104,7 +104,7 @@ public class KiWiStoreProvider implements StoreProvider {
         String dbUser  = configurationService.getStringConfiguration("database.user");
         String dbPass  = configurationService.getStringConfiguration("database.password");
 
-        KiWiConfiguration configuration = new KiWiConfiguration("lmf", jdbcUrl, dbUser, dbPass, dialect, configurationService.getDefaultContext(), configurationService.getInferredContext());
+        KiWiConfiguration configuration = new KiWiConfiguration(configurationService.getStringConfiguration("clustering.name", "Marmotta") + " KiWi", jdbcUrl, dbUser, dbPass, dialect, configurationService.getDefaultContext(), configurationService.getInferredContext());
         configuration.setQueryLoggingEnabled(configurationService.getBooleanConfiguration(DEBUG_SLOWQUERIES, false));
         configuration.setTripleBatchCommit(configurationService.getBooleanConfiguration("database.triples.batchcommit", true));
         configuration.setTripleBatchSize(configurationService.getIntConfiguration("database.triples.batchsize", 10000));

http://git-wip-us.apache.org/repos/asf/marmotta/blob/01c33dc3/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
index 459d63c..fdb7288 100644
--- a/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
+++ b/platform/marmotta-core/src/main/java/org/apache/marmotta/platform/core/services/cache/CachingServiceImpl.java
@@ -31,6 +31,7 @@ import org.infinispan.distribution.ch.SyncConsistentHashFactory;
 import org.infinispan.eviction.EvictionStrategy;
 import org.infinispan.manager.DefaultCacheManager;
 import org.infinispan.manager.EmbeddedCacheManager;
+import org.infinispan.remoting.transport.Address;
 import org.slf4j.Logger;
 
 import javax.annotation.PostConstruct;
@@ -82,7 +83,7 @@ public class CachingServiceImpl implements CachingService {
             globalConfiguration = new GlobalConfigurationBuilder()
                     .transport()
                         .defaultTransport()
-                        .clusterName(configurationService.getStringConfiguration("clustering.name", "Marmotta"))
+                        .clusterName(configurationService.getStringConfiguration("clustering.name", "Marmotta") + " Platform")
                         .machineId(configurationService.getServerName())
                         .addProperty("configurationFile", "jgroups-marmotta.xml")
                     .globalJmxStatistics()
@@ -128,6 +129,13 @@ public class CachingServiceImpl implements CachingService {
 
 
         cacheManager = new DefaultCacheManager(globalConfiguration, defaultConfiguration, true);
+
+        if(log.isInfoEnabled()) {
+            log.info("Members in Apache Marmotta cache cluster:");
+            for(Address a : cacheManager.getMembers()) {
+                log.info(" - {}",a);
+            }
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/marmotta/blob/01c33dc3/platform/marmotta-core/src/main/resources/jgroups-marmotta.xml
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/main/resources/jgroups-marmotta.xml b/platform/marmotta-core/src/main/resources/jgroups-marmotta.xml
index aa5ce8c..8529c45 100644
--- a/platform/marmotta-core/src/main/resources/jgroups-marmotta.xml
+++ b/platform/marmotta-core/src/main/resources/jgroups-marmotta.xml
@@ -1,9 +1,26 @@
+<!--
+  ~ 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.
+  -->
+
 <config xmlns="urn:org:jgroups"
         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
         xsi:schemaLocation="urn:org:jgroups http://www.jgroups.org/schema/JGroups-3.4.xsd">
    <UDP
          mcast_addr="${jgroups.udp.mcast_addr:228.6.7.8}"
-         mcast_port="${jgroups.udp.mcast_port:46655}"
+         mcast_port="${jgroups.udp.mcast_port:46656}"
          tos="8"
          ucast_recv_buf_size="20m"
          ucast_send_buf_size="640k"

http://git-wip-us.apache.org/repos/asf/marmotta/blob/01c33dc3/platform/marmotta-core/src/test/resources/ehcache-marmotta.xml
----------------------------------------------------------------------
diff --git a/platform/marmotta-core/src/test/resources/ehcache-marmotta.xml b/platform/marmotta-core/src/test/resources/ehcache-marmotta.xml
deleted file mode 100644
index 3c754d2..0000000
--- a/platform/marmotta-core/src/test/resources/ehcache-marmotta.xml
+++ /dev/null
@@ -1,405 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-   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.
--->
-
-<!--
-CacheManager Configuration
-==========================
-An ehcache-marmotta.xml corresponds to a single CacheManager.
-
-See instructions below or the ehcache schema (ehcache.xsd) on how to configure.
-
-System property tokens can be specified in this file which are replaced when the configuration
-is loaded. For example multicastGroupPort=${multicastGroupPort} can be replaced with the
-System property either from an environment variable or a system property specified with a
-command line switch such as -DmulticastGroupPort=4446.
-
-The attributes of <ehcache> are:
-* name - an optional name for the CacheManager.  The name is optional and primarily used 
-for documentation or to distinguish Terracotta clustered cache state.  With Terracotta 
-clustered caches, a combination of CacheManager name and cache name uniquely identify a 
-particular cache store in the Terracotta clustered memory.
-* updateCheck - an optional boolean flag specifying whether this CacheManager should check
-for new versions of Ehcache over the Internet.  If not specified, updateCheck="true".
-* monitoring - an optional setting that determines whether the CacheManager should 
-automatically register the SampledCacheMBean with the system MBean server.  Currently,
-this monitoring is only useful when using Terracotta and thus the "autodetect" value 
-will detect the presence of Terracotta and register the MBean.  Other allowed values 
-are "on" and "off".  The default is "autodetect".
--->    
-<ehcache xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:noNamespaceSchemaLocation="http://ehcache.org/ehcache.xsd" name="lmf">
-
-    <!-- 
-    DiskStore configuration
-    =======================
-
-    The diskStore element is optional. To turn off disk store path creation, comment out the diskStore
-    element below.
-
-    Configure it if you have overflowToDisk or diskPersistent enabled for any cache.
-
-    If it is not configured, and a cache is created which requires a disk store, a warning will be
-     issued and java.io.tmpdir will automatically be used.
-
-    diskStore has only one attribute - "path". It is the path to the directory where
-    .data and .index files will be created.
-
-    If the path is one of the following Java System Property it is replaced by its value in the
-    running VM. For backward compatibility these are not specified without being enclosed in the ${token}
-    replacement syntax.
-
-    The following properties are translated:
-    * user.home - KiWiUser's home directory
-    * user.dir - KiWiUser's current working directory
-    * java.io.tmpdir - Default temp file path
-    * ehcache.disk.store.dir - A system property you would normally specify on the command line
-      e.g. java -Dehcache.disk.store.dir=/u01/myapp/diskdir ...
-
-    Subdirectories can be specified below the property e.g. java.io.tmpdir/one
-
-    -->
-    <!--
-    <diskStore path="java.io.tmpdir/lmf-cache/lmf"/>
-
-    -->
-
-   <!--
-    Cachemanagereventlistener
-    =========================
-    Specifies a CacheManagerEventListenerFactory which is notified when Caches are added
-    or removed from the CacheManager.
-
-    The attributes of CacheManagerEventListenerFactory are:
-    * class - a fully qualified factory class name
-    * properties - comma separated properties having meaning only to the factory.
-
-    Sets the fully qualified class name to be registered as the CacheManager event listener.
-
-    The events include:
-    * adding a Cache
-    * removing a Cache
-
-    Callbacks to listener methods are synchronous and unsynchronized. It is the responsibility
-    of the implementer to safely handle the potential performance and thread safety issues
-    depending on what their listener is doing.
-
-    If no class is specified, no listener is created. There is no default.
-    -->
-    <cacheManagerEventListenerFactory class="" properties=""/>
-
-
-    <!--
-    CacheManagerPeerProvider
-    ========================
-    (For distributed operation)
-
-    Specifies a CacheManagerPeerProviderFactory which will be used to create a
-    CacheManagerPeerProvider, which discovers other CacheManagers in the cluster.
-
-    One or more providers can be configured. The first one in the ehcache-marmotta.xml is the default, which is used
-    for replication and bootstrapping.
-
-    The attributes of cacheManagerPeerProviderFactory are:
-    * class - a fully qualified factory class name
-    * properties - comma separated properties having meaning only to the factory.
-
-    Providers are available for RMI, JGroups and JMS as shown following.
-
-    RMICacheManagerPeerProvider
-    +++++++++++++++++++++++++++
-
-    Ehcache comes with a built-in RMI-based distribution system with two means of discovery of
-    CacheManager peers participating in the cluster:
-    * automatic, using a multicast group. This one automatically discovers peers and detects
-      changes such as peers entering and leaving the group
-    * manual, using manual rmiURL configuration. A hardcoded list of peers is provided at
-      configuration time.
-
-    Configuring Automatic Discovery:
-    Automatic discovery is configured as per the following example:
-    <cacheManagerPeerProviderFactory
-                        class="net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                        properties="hostName=fully_qualified_hostname_or_ip,
-                                    peerDiscovery=automatic, multicastGroupAddress=230.0.0.1,
-                                    multicastGroupPort=4446, timeToLive=32"/>
-
-    Valid properties are:
-    * peerDiscovery (mandatory) - specify "automatic"
-    * multicastGroupAddress (mandatory) - specify a valid multicast group address
-    * multicastGroupPort (mandatory) - specify a dedicated port for the multicast heartbeat
-      traffic
-    * timeToLive - specify a value between 0 and 255 which determines how far the packets will
-      propagate.
-
-      By convention, the restrictions are:
-      0   - the same host
-      1   - the same subnet
-      32  - the same site
-      64  - the same region
-      128 - the same continent
-      255 - unrestricted
-
-     * hostName - the hostname or IP of the interface to be used for sending and receiving multicast packets
-       (relevant to mulithomed hosts only)
-
-    Configuring Manual Discovery:
-    Manual discovery requires a unique configuration per host. It is contains a list of rmiURLs for the peers, other
-    than itself. So, if we have server1, server2 and server3 the configuration will be:
-
-    In server1's configuration:
-    <cacheManagerPeerProviderFactory class=
-                          "net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                          properties="peerDiscovery=manual,
-                          rmiUrls=//server2:40000/sampleCache1|//server3:40000/sampleCache1
-                          | //server2:40000/sampleCache2|//server3:40000/sampleCache2"
-                          propertySeparator="," />
-
-    In server2's configuration:
-    <cacheManagerPeerProviderFactory class=
-                          "net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                          properties="peerDiscovery=manual,
-                          rmiUrls=//server1:40000/sampleCache1|//server3:40000/sampleCache1
-                          | //server1:40000/sampleCache2|//server3:40000/sampleCache2"
-                          propertySeparator="," />
-
-    In server3's configuration:
-    <cacheManagerPeerProviderFactory class=
-                          "net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"
-                          properties="peerDiscovery=manual,
-                          rmiUrls=//server1:40000/sampleCache1|//server2:40000/sampleCache1
-                          | //server1:40000/sampleCache2|//server2:40000/sampleCache2"
-                          propertySeparator="," />
-
-
-    Valid properties are:
-    * peerDiscovery (mandatory) - specify "manual"
-    * rmiUrls (mandatory) - specify a pipe separated list of rmiUrls, in the form
-                            //hostname:port
-    * hostname (optional) - the hostname is the hostname of the remote CacheManager peer. The port is the listening
-      port of the RMICacheManagerPeerListener of the remote CacheManager peer.
-    
-    JGroupsCacheManagerPeerProvider
-    +++++++++++++++++++++++++++++++
-    <cacheManagerPeerProviderFactory class="net.sf.ehcache.distribution.jgroups.JGroupsCacheManagerPeerProviderFactory"
-                                     properties="connect=UDP(mcast_addr=231.12.21.132;mcast_port=45566;ip_ttl=32;
-                                     mcast_send_buf_size=150000;mcast_recv_buf_size=80000):
-                                     PING(timeout=2000;num_initial_members=6):
-                                     MERGE2(min_interval=5000;max_interval=10000):
-                                     FD_SOCK:VERIFY_SUSPECT(timeout=1500):
-                                     pbcast.NAKACK(gc_lag=10;retransmit_timeout=3000):
-                                     UNICAST(timeout=5000):
-                                     pbcast.STABLE(desired_avg_gossip=20000):
-                                     FRAG:
-                                     pbcast.GMS(join_timeout=5000;join_retry_timeout=2000;shun=false;print_local_addr=false)"
-                                     propertySeparator="::"
-            />
-     The only property necessary is the connect String used by jgroups to configure itself. Refer to the Jgroups documentation for explanation
-     of all the protocols. The example above uses UDP multicast. If the connect property is not specified the default JGroups connection will be
-     used.       
-
-
-    JMSCacheManagerPeerProviderFactory
-    ++++++++++++++++++++++++++++++++++
-    <cacheManagerPeerProviderFactory
-            class="net.sf.ehcache.distribution.jms.JMSCacheManagerPeerProviderFactory"
-            properties="..."
-            propertySeparator=","
-            />
-
-    The JMS PeerProviderFactory uses JNDI to maintain message queue independence. Refer to the manual for full configuration
-    examples using ActiveMQ and Open Message Queue.
-
-    Valid properties are:
-    * initialContextFactoryName (mandatory) - the name of the factory used to create the message queue initial context.
-    * providerURL (mandatory) - the JNDI configuration information for the service provider to use.
-    * topicConnectionFactoryBindingName (mandatory) - the JNDI binding name for the TopicConnectionFactory
-    * topicBindingName (mandatory) - the JNDI binding name for the topic name
-    * getQueueBindingName (mandatory only if using jmsCacheLoader) - the JNDI binding name for the queue name
-    * securityPrincipalName - the JNDI java.naming.security.principal
-    * securityCredentials - the JNDI java.naming.security.credentials
-    * urlPkgPrefixes - the JNDI java.naming.factory.url.pkgs
-    * userName - the user name to use when creating the TopicConnection to the Message Queue
-    * password - the password to use when creating the TopicConnection to the Message Queue
-    * acknowledgementMode - the JMS Acknowledgement mode for both publisher and subscriber. The available choices are
-                            AUTO_ACKNOWLEDGE, DUPS_OK_ACKNOWLEDGE and SESSION_TRANSACTED. The default is AUTO_ACKNOWLEDGE.
-    -->
-<!--    <cacheManagerPeerProviderFactory-->
-<!--            class="net.sf.ehcache.distribution.RMICacheManagerPeerProviderFactory"-->
-<!--            properties="peerDiscovery=automatic,-->
-<!--                        multicastGroupAddress=230.0.0.1,-->
-<!--                        multicastGroupPort=4446, timeToLive=1"-->
-<!--            propertySeparator=","-->
-<!--            />-->
-
-
-    <!--
-    CacheManagerPeerListener
-    ========================
-    (Enable for distributed operation)
-
-    Specifies a CacheManagerPeerListenerFactory which will be used to create a
-    CacheManagerPeerListener, which listens for messages from cache replicators participating in the cluster.
-
-    The attributes of cacheManagerPeerListenerFactory are:
-    class - a fully qualified factory class name
-    properties - comma separated properties having meaning only to the factory.
-
-    Ehcache comes with a built-in RMI-based distribution system. The listener component is
-    RMICacheManagerPeerListener which is configured using
-    RMICacheManagerPeerListenerFactory. It is configured as per the following example:
-
-    <cacheManagerPeerListenerFactory
-        class="net.sf.ehcache.distribution.RMICacheManagerPeerListenerFactory"
-        properties="hostName=fully_qualified_hostname_or_ip,
-                    port=40001,
-                    remoteObjectPort=40002,
-                    socketTimeoutMillis=120000"
-                    propertySeparator="," />
-
-    All properties are optional. They are:
-    * hostName - the hostName of the host the listener is running on. Specify
-      where the host is multihomed and you want to control the interface over which cluster
-      messages are received. Defaults to the host name of the default interface if not
-      specified.
-    * port - the port the RMI Registry listener listens on. This defaults to a free port if not specified.
-    * remoteObjectPort - the port number on which the remote objects bound in the registry receive calls.
-                         This defaults to a free port if not specified.
-    * socketTimeoutMillis - the number of ms client sockets will stay open when sending
-      messages to the listener. This should be long enough for the slowest message.
-      If not specified it defaults to 120000ms.
-
-    -->
-<!--    <cacheManagerPeerListenerFactory-->
-<!--            class="net.sf.ehcache.distribution.RMICacheManagerPeerListenerFactory"/>-->
-
-    <!-- Cache configuration.
-
-    The following attributes are required.
-
-    name:
-    Sets the name of the cache. This is used to identify the cache. It must be unique.
-
-    maxElementsInMemory:
-    Sets the maximum number of objects that will be created in memory
-
-        maxElementsOnDisk:
-    Sets the maximum number of objects that will be maintained in the DiskStore
-        The default value is zero, meaning unlimited.
-
-    eternal:
-    Sets whether elements are eternal. If eternal,  timeouts are ignored and the
-    element is never expired.
-
-    overflowToDisk:
-    Sets whether elements can overflow to disk when the memory store
-    has reached the maxInMemory limit.
-
-    The following attributes are optional.
-
-    timeToIdleSeconds:
-    Sets the time to idle for an element before it expires.
-    i.e. The maximum amount of time between accesses before an element expires
-    Is only used if the element is not eternal.
-    Optional attribute. A value of 0 means that an Element can idle for infinity.
-    The default value is 0.
-
-    timeToLiveSeconds:
-    Sets the time to live for an element before it expires.
-    i.e. The maximum time between creation time and when an element expires.
-    Is only used if the element is not eternal.
-    Optional attribute. A value of 0 means that and Element can live for infinity.
-    The default value is 0.
-
-    diskPersistent:
-    Whether the disk store persists between restarts of the Virtual Machine.
-    The default value is false.
-
-    diskExpiryThreadIntervalSeconds:
-    The number of seconds between runs of the disk expiry thread. The default value
-    is 120 seconds.
-
-    memoryStoreEvictionPolicy:
-    Policy would be enforced upon reaching the maxElementsInMemory limit. Default
-    policy is Least Recently Used (specified as LRU). Other policies available -
-    First In First Out (specified as FIFO) and Less Frequently Used
-    (specified as LFU)
-
-    -->
-
-    <!--
-    Mandatory Default Cache configuration. These settings will be applied to caches
-    created programmtically using CacheManager.add(String cacheName)
-    -->
-    <defaultCache
-            maxElementsInMemory="20000"
-            overflowToDisk="false"
-            memoryStoreEvictionPolicy="LRU"
-            />
-
-
-    <!-- the cache used for triple queries by KiWi -->
-    <cache name="uri-node-cache"
-           statistics="true"
-           maxElementsInMemory="100000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-    <cache name="anon-node-cache"
-           statistics="true"
-           maxElementsInMemory="10000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-    <cache name="literal-cache"
-           statistics="true"
-           maxElementsInMemory="10000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-
-    <cache name="namespace-prefix-cache"
-           statistics="true"
-           maxElementsInMemory="100"
-           overflowToDisk="false"/>
-
-    <cache name="namespace-uri-cache"
-           statistics="true"
-           maxElementsInMemory="100"
-           overflowToDisk="false"/>
-
-    <!-- the cache used for triple queries by KiWi -->
-    <cache name="query-cache"
-           statistics="true"
-           maxElementsInMemory="200000"
-           timeToIdleSeconds="3600"
-           overflowToDisk="false"/>
-
-    <!-- the cache used for resource lookups from module jar files -->
-    <cache name="resource-cache"
-           statistics="true"
-           maxElementsInMemory="10000"
-           timeToIdleSeconds="3600"
-           timeToLiveSeconds="3600"
-           overflowToDisk="false"
-           memoryStoreEvictionPolicy="LRU"/>
-
-
-    <!--  uncomment to enable cache debugging -->
-<!-- 
-	<cacheManagerPeerListenerFactory
-	    class="org.terracotta.ehcachedx.monitor.probe.ProbePeerListenerFactory"
-	    properties="monitorAddress=localhost, monitorPort=9889" />
--->
-
-</ehcache>


[7/7] git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/marmotta into develop

Posted by ss...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/marmotta into develop


Project: http://git-wip-us.apache.org/repos/asf/marmotta/repo
Commit: http://git-wip-us.apache.org/repos/asf/marmotta/commit/92205ec4
Tree: http://git-wip-us.apache.org/repos/asf/marmotta/tree/92205ec4
Diff: http://git-wip-us.apache.org/repos/asf/marmotta/diff/92205ec4

Branch: refs/heads/develop
Commit: 92205ec473ba30548338f50942857e3bbbeb3b3a
Parents: 01c33dc 593ea4f
Author: Sebastian Schaffert <ss...@apache.org>
Authored: Tue Dec 17 15:46:58 2013 +0100
Committer: Sebastian Schaffert <ss...@apache.org>
Committed: Tue Dec 17 15:46:58 2013 +0100

----------------------------------------------------------------------
 .../core/api/triplestore/ContextService.java    |  7 ++
 .../triplestore/ContextServiceImpl.java         | 21 +++++
 .../triplestore/ContextWebService.java          |  5 +-
 .../src/main/resources/web/admin/contexts.html  | 90 ++++++++++++++------
 .../src/main/resources/web/admin/ldpath.html    | 13 +--
 .../src/main/resources/web/admin/style.css      | 12 ++-
 .../src/main/resources/web/admin/configure.html | 24 ++++--
 .../resources/web/admin/widget/conf_reasoner.js | 56 ++++++++++--
 8 files changed, 177 insertions(+), 51 deletions(-)
----------------------------------------------------------------------



[2/7] git commit: towards MARMOTTA-388: triplestore now working with infinispan, configurable cluster mode

Posted by ss...@apache.org.
towards MARMOTTA-388: triplestore now working with infinispan, configurable cluster mode


Project: http://git-wip-us.apache.org/repos/asf/marmotta/repo
Commit: http://git-wip-us.apache.org/repos/asf/marmotta/commit/0d79835c
Tree: http://git-wip-us.apache.org/repos/asf/marmotta/tree/0d79835c
Diff: http://git-wip-us.apache.org/repos/asf/marmotta/diff/0d79835c

Branch: refs/heads/develop
Commit: 0d79835c1e27333ba72229391a8303ba94b4c996
Parents: c0c795b
Author: Sebastian Schaffert <ss...@apache.org>
Authored: Mon Dec 16 19:48:58 2013 +0100
Committer: Sebastian Schaffert <ss...@apache.org>
Committed: Mon Dec 16 19:48:58 2013 +0100

----------------------------------------------------------------------
 .../kiwi/loader/generic/KiWiHandler.java        |  46 +-
 .../marmotta/kiwi/caching/KiWiCacheManager.java | 110 ++-
 .../marmotta/kiwi/config/KiWiConfiguration.java |  41 +
 .../marmotta/kiwi/config/RegistryStrategy.java  |  39 +
 .../registry/CacheTripleRegistry.java           |  49 +-
 .../marmotta/kiwi/sail/KiWiValueFactory.java    |  16 +-
 .../src/main/resources/ehcache-kiwi.xml         | 770 -------------------
 .../src/test/resources/logback.xml              |   2 +-
 8 files changed, 225 insertions(+), 848 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/marmotta/blob/0d79835c/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java b/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java
index 244d660..b9ee1f1 100644
--- a/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java
+++ b/libraries/kiwi/kiwi-loader/src/main/java/org/apache/marmotta/kiwi/loader/generic/KiWiHandler.java
@@ -3,9 +3,6 @@ package org.apache.marmotta.kiwi.loader.generic;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
-import net.sf.ehcache.Element;
-import net.sf.ehcache.constructs.blocking.CacheEntryFactory;
-import net.sf.ehcache.constructs.blocking.SelfPopulatingCache;
 import org.apache.marmotta.commons.sesame.model.Namespaces;
 import org.apache.marmotta.commons.sesame.tripletable.IntArray;
 import org.apache.marmotta.commons.util.DateUtils;
@@ -53,9 +50,6 @@ public class KiWiHandler implements RDFHandler {
 
     protected KiWiLoaderConfiguration config;
 
-    protected SelfPopulatingCache literalCache;
-    protected SelfPopulatingCache uriCache;
-    protected SelfPopulatingCache bnodeCache;
     protected LoadingCache<String,Locale> localeCache;
 
     // if non-null, all imported statements will have this context (regardless whether they specified a different context)
@@ -75,27 +69,6 @@ public class KiWiHandler implements RDFHandler {
         this.config     = config;
         this.store      = store;
 
-        this.literalCache = new SelfPopulatingCache(store.getPersistence().getCacheManager().getLoaderCache(), new CacheEntryFactory() {
-            @Override
-            public Object createEntry(Object key) throws Exception {
-                return createLiteral((Literal)key);
-            }
-        });
-
-        this.uriCache = new SelfPopulatingCache(store.getPersistence().getCacheManager().getLoaderCache(), new CacheEntryFactory() {
-            @Override
-            public Object createEntry(Object key) throws Exception {
-                return createURI(((URI)key).stringValue());
-            }
-        });
-
-        this.bnodeCache = new SelfPopulatingCache(store.getPersistence().getCacheManager().getLoaderCache(), new CacheEntryFactory() {
-            @Override
-            public Object createEntry(Object key) throws Exception {
-                return createBNode(((BNode)key).stringValue());
-            }
-        });
-
         this.localeCache = CacheBuilder.newBuilder()
                 .maximumSize(100)
                 .build(new CacheLoader<String, Locale>() {
@@ -295,27 +268,14 @@ public class KiWiHandler implements RDFHandler {
         } else if(value instanceof KiWiNode) {
             return (KiWiNode)value;
         } else if(value instanceof URI) {
-            Element e = uriCache.get((URI) value);
-            if(e != null) {
-                return (KiWiNode) e.getObjectValue();
-            }
+            return createURI(((URI)value).stringValue());
         } else if(value instanceof BNode) {
-            Element e = bnodeCache.get(((BNode)value));
-            if(e != null) {
-                return (KiWiNode) e.getObjectValue();
-            }
+            return createBNode(((BNode)value).stringValue());
         } else if(value instanceof Literal) {
-            Literal l = (Literal)value;
-            Element e = literalCache.get(l);
-            if(e != null) {
-                return (KiWiNode) e.getObjectValue();
-            }
-
+            return createLiteral((Literal)value);
         } else {
             throw new IllegalArgumentException("the value passed as argument does not have the correct type");
         }
-        throw new IllegalStateException("could not construct or load node");
-
     }
 
     protected KiWiLiteral createLiteral(Literal l) throws ExecutionException {

http://git-wip-us.apache.org/repos/asf/marmotta/blob/0d79835c/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
index 1c444a4..7be150c 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/caching/KiWiCacheManager.java
@@ -49,6 +49,8 @@ public class KiWiCacheManager {
     public static final String NAMESPACE_URI_CACHE = "namespace-uri-cache";
     public static final String NAMESPACE_PREFIX_CACHE = "namespace-prefix-cache";
     public static final String LOADER_CACHE = "loader-cache";
+    public static final String REGISTRY_CACHE = "registry-cache";
+
     private EmbeddedCacheManager cacheManager;
 
     private GlobalConfiguration globalConfiguration;
@@ -56,33 +58,58 @@ public class KiWiCacheManager {
     // default configuration: distributed cache, 100000 entries, 300 seconds expiration, 60 seconds idle
     private Configuration defaultConfiguration;
 
+    private boolean clustered;
+
     public KiWiCacheManager(KiWiConfiguration config) {
-        globalConfiguration = new GlobalConfigurationBuilder()
-                .transport()
-                    .defaultTransport()
-                    .clusterName(config.getName())
-                    .machineId("instance-" + config.getDatacenterId())
-                    .addProperty("configurationFile", "jgroups-udp.xml")
-                .build();
-
-
-        defaultConfiguration = new ConfigurationBuilder()
-                .clustering()
-                    .cacheMode(CacheMode.DIST_SYNC)
-                    .sync()
-                    .l1()
-                        .lifespan(25, TimeUnit.SECONDS)
-                    .hash()
-                        .numOwners(2)
-                        .numSegments(100)
-                        .consistentHashFactory(new SyncConsistentHashFactory())
-                .eviction()
-                    .strategy(EvictionStrategy.LIRS)
-                    .maxEntries(100000)
-                .expiration()
-                    .lifespan(5, TimeUnit.MINUTES)
-                    .maxIdle(1, TimeUnit.MINUTES)
-                .build();
+
+        this.clustered = config.isClustered();
+
+        if(clustered) {
+            globalConfiguration = new GlobalConfigurationBuilder()
+                    .transport()
+                        .defaultTransport()
+                        .clusterName(config.getName())
+                        .machineId("instance-" + config.getDatacenterId())
+                        .addProperty("configurationFile", "jgroups-udp.xml")
+                    .globalJmxStatistics()
+                    .build();
+
+
+            defaultConfiguration = new ConfigurationBuilder()
+                    .clustering()
+                        .cacheMode(CacheMode.DIST_ASYNC)
+                        .async()
+                        .l1()
+                            .lifespan(25, TimeUnit.SECONDS)
+                        .hash()
+                            .numOwners(2)
+                            .numSegments(100)
+                            .consistentHashFactory(new SyncConsistentHashFactory())
+                    .eviction()
+                        .strategy(EvictionStrategy.LIRS)
+                        .maxEntries(100000)
+                    .expiration()
+                        .lifespan(5, TimeUnit.MINUTES)
+                        .maxIdle(1, TimeUnit.MINUTES)
+                    .build();
+        } else {
+            globalConfiguration = new GlobalConfigurationBuilder()
+                    .globalJmxStatistics()
+                    .build();
+
+            defaultConfiguration = new ConfigurationBuilder()
+                    .clustering()
+                        .cacheMode(CacheMode.LOCAL)
+                    .eviction()
+                        .strategy(EvictionStrategy.LIRS)
+                        .maxEntries(100000)
+                    .expiration()
+                        .lifespan(5, TimeUnit.MINUTES)
+                        .maxIdle(1, TimeUnit.MINUTES)
+                    .build();
+
+        }
+
 
         cacheManager = new DefaultCacheManager(globalConfiguration, defaultConfiguration, true);
     }
@@ -233,6 +260,37 @@ public class KiWiCacheManager {
 
 
     /**
+     * Create and return the cache used by the CacheTripleRegistry. This is an unlimited synchronous replicated
+     * cache and should be used with care.
+     * @return
+     */
+    public Cache getRegistryCache() {
+        if(!cacheManager.cacheExists(LOADER_CACHE)) {
+            if(clustered) {
+                Configuration registryConfiguration = new ConfigurationBuilder()
+                    .clustering()
+                        .cacheMode(CacheMode.REPL_SYNC)
+                        .sync()
+                        .l1()
+                            .lifespan(25, TimeUnit.SECONDS)
+                    .eviction()
+                        .strategy(EvictionStrategy.NONE)
+                    .build();
+                cacheManager.defineConfiguration(REGISTRY_CACHE, registryConfiguration);
+            } else {
+                Configuration registryConfiguration = new ConfigurationBuilder()
+                    .clustering()
+                        .cacheMode(CacheMode.LOCAL)
+                    .eviction()
+                        .strategy(EvictionStrategy.NONE)
+                    .build();
+                cacheManager.defineConfiguration(REGISTRY_CACHE, registryConfiguration);
+            }
+        }
+        return cacheManager.getCache(REGISTRY_CACHE);
+    }
+
+    /**
      * Get the cache with the given name from the cache manager. Can be used to request additional
      * caches from the cache manager that are not covered by explicit methods.
      *

http://git-wip-us.apache.org/repos/asf/marmotta/blob/0d79835c/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/config/KiWiConfiguration.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/config/KiWiConfiguration.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/config/KiWiConfiguration.java
index 65019ba..0a06aa0 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/config/KiWiConfiguration.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/config/KiWiConfiguration.java
@@ -100,6 +100,14 @@ public class KiWiConfiguration {
     private int datacenterId = 0;
 
 
+    private RegistryStrategy registryStrategy = RegistryStrategy.CACHE;
+
+    /**
+     * Enable to turn on cluster mode (e.g. for cache replication)
+     */
+    private boolean clustered = false;
+
+
     public KiWiConfiguration(String name, String jdbcUrl, String dbUser, String dbPassword, KiWiDialect dialect) {
         this(name, jdbcUrl, dbUser, dbPassword, dialect, null, null);
     }
@@ -321,4 +329,37 @@ public class KiWiConfiguration {
     public void setDatacenterId(int datacenterId) {
         this.datacenterId = datacenterId;
     }
+
+
+    /**
+     * Return the strategy to use for synchronizing between transactions. Currently either "database" or "cache".
+     * @return
+     */
+    public RegistryStrategy getRegistryStrategy() {
+        return registryStrategy;
+    }
+
+    /**
+     * Change the strategy to use for synchronizing between transactions. Currently either "database" or "cache".
+     * @return
+     */
+    public void setRegistryStrategy(RegistryStrategy registryStrategy) {
+        this.registryStrategy = registryStrategy;
+    }
+
+    /**
+     * If true, support for special cluster features is enabled (e.g. replicated caches).
+     *
+     * @return
+     */
+    public boolean isClustered() {
+        return clustered;
+    }
+
+    /**
+     * If true, support for special cluster features is enabled (e.g. replicated caches).
+     */
+    public void setClustered(boolean clustered) {
+        this.clustered = clustered;
+    }
 }

http://git-wip-us.apache.org/repos/asf/marmotta/blob/0d79835c/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/config/RegistryStrategy.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/config/RegistryStrategy.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/config/RegistryStrategy.java
new file mode 100644
index 0000000..841d1cf
--- /dev/null
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/config/RegistryStrategy.java
@@ -0,0 +1,39 @@
+/*
+ * 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.marmotta.kiwi.config;
+
+/**
+ * The strategy to use for transactional triple registry. This is needed to avoid two parallel transactions creating
+ * the same triple with different IDs.
+ *
+ * @author Sebastian Schaffert (sschaffert@apache.org)
+ */
+public enum RegistryStrategy {
+
+    /**
+     * Use the database to synchronize between several parallel instances. Slow but requires less memory.
+     */
+    DATABASE,
+
+    /**
+     * Use a synchronized replicated infinispan cache to synchronize between parallel instances. Faster but requires
+     * more memory.
+     */
+    CACHE
+
+}

http://git-wip-us.apache.org/repos/asf/marmotta/blob/0d79835c/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/CacheTripleRegistry.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/CacheTripleRegistry.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/CacheTripleRegistry.java
index cb1896c..e35744e 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/CacheTripleRegistry.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/persistence/registry/CacheTripleRegistry.java
@@ -18,19 +18,37 @@
 package org.apache.marmotta.kiwi.persistence.registry;
 
 import org.apache.marmotta.commons.sesame.tripletable.IntArray;
-import org.infinispan.AdvancedCache;
+import org.apache.marmotta.kiwi.caching.KiWiCacheManager;
+import org.infinispan.Cache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 /**
- * A triple registry implementation based on the Infinispan cache.
+ * A triple registry implementation based on the Infinispan cache. Registry entries are stored in a replicated,
+ * synchronized Infinispan cache. Transaction information is kept locally.
  *
  * @author Sebastian Schaffert (sschaffert@apache.org)
  */
 public class CacheTripleRegistry implements KiWiTripleRegistry {
 
+    private static Logger log = LoggerFactory.getLogger(CacheTripleRegistry.class);
 
-    AdvancedCache<IntArray,Map<Long,Long>> cache;
+    private Cache<Long,Long> cache;
+
+
+    private Map<Long,List<Long>>  transactions;
+
+
+    public CacheTripleRegistry(KiWiCacheManager cacheManager) {
+        cache        = cacheManager.getRegistryCache();
+        transactions = new HashMap<>();
+
+    }
 
 
     /**
@@ -42,7 +60,17 @@ public class CacheTripleRegistry implements KiWiTripleRegistry {
      */
     @Override
     public void registerKey(IntArray key, long transactionId, long tripleId) {
+        List<Long> transaction = transactions.get(transactionId);
+        if(transaction == null) {
+            transaction = new ArrayList<>();
+            transactions.put(transactionId, transaction);
+        }
+        Long old = cache.put(key.longHashCode(),tripleId);
+        transaction.add(key.longHashCode());
 
+        if(old != null && old != tripleId) {
+            log.warn("registered a new triple ID for an already existing triple");
+        }
     }
 
     /**
@@ -54,7 +82,12 @@ public class CacheTripleRegistry implements KiWiTripleRegistry {
      */
     @Override
     public long lookupKey(IntArray key) {
-        return 0;
+        Long value = cache.get(key.longHashCode());
+        if(value != null) {
+            return value;
+        } else {
+            return -1;
+        }
     }
 
     /**
@@ -65,7 +98,11 @@ public class CacheTripleRegistry implements KiWiTripleRegistry {
      */
     @Override
     public void releaseTransaction(long transactionId) {
-
+        if(transactions.containsKey(transactionId)) {
+            for(long key : transactions.remove(transactionId)) {
+                cache.removeAsync(key);
+            }
+        }
     }
 
     /**
@@ -75,6 +112,6 @@ public class CacheTripleRegistry implements KiWiTripleRegistry {
      */
     @Override
     public void deleteKey(IntArray key) {
-
+        cache.remove(key.longHashCode());
     }
 }

http://git-wip-us.apache.org/repos/asf/marmotta/blob/0d79835c/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java
index 444af01..f248dfb 100644
--- a/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java
+++ b/libraries/kiwi/kiwi-triplestore/src/main/java/org/apache/marmotta/kiwi/sail/KiWiValueFactory.java
@@ -23,7 +23,9 @@ import org.apache.marmotta.commons.sesame.tripletable.IntArray;
 import org.apache.marmotta.commons.util.DateUtils;
 import org.apache.marmotta.kiwi.model.rdf.*;
 import org.apache.marmotta.kiwi.persistence.KiWiConnection;
+import org.apache.marmotta.kiwi.persistence.registry.CacheTripleRegistry;
 import org.apache.marmotta.kiwi.persistence.registry.DBTripleRegistry;
+import org.apache.marmotta.kiwi.persistence.registry.KiWiTripleRegistry;
 import org.openrdf.model.*;
 import org.openrdf.model.impl.ContextStatementImpl;
 import org.slf4j.Logger;
@@ -50,13 +52,23 @@ public class KiWiValueFactory implements ValueFactory {
 
     private KiWiStore store;
 
-    private DBTripleRegistry registry;
+    private KiWiTripleRegistry registry;
 
     private String defaultContext;
 
     public KiWiValueFactory(KiWiStore store, String defaultContext) {
         anonIdGenerator = new Random();
-        registry        = new DBTripleRegistry(store);
+
+        switch (store.getPersistence().getConfiguration().getRegistryStrategy()) {
+            case DATABASE:
+                registry        = new DBTripleRegistry(store);
+                break;
+            case CACHE:
+                registry        = new CacheTripleRegistry(store.getPersistence().getCacheManager());
+                break;
+            default:
+                registry        = new CacheTripleRegistry(store.getPersistence().getCacheManager());
+        }
 
         this.store          = store;
         this.defaultContext = defaultContext;

http://git-wip-us.apache.org/repos/asf/marmotta/blob/0d79835c/libraries/kiwi/kiwi-triplestore/src/main/resources/ehcache-kiwi.xml
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/main/resources/ehcache-kiwi.xml b/libraries/kiwi/kiwi-triplestore/src/main/resources/ehcache-kiwi.xml
deleted file mode 100644
index ccbd3fe..0000000
--- a/libraries/kiwi/kiwi-triplestore/src/main/resources/ehcache-kiwi.xml
+++ /dev/null
@@ -1,770 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-
-    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.
-
--->
-<!--
-CacheManager Configuration
-==========================
-An ehcache.xml corresponds to a single CacheManager.
-
-See instructions below or the ehcache schema (ehcache.xsd) on how to configure.
-
-System property tokens can be specified in this file which are replaced when the configuration
-is loaded. For example multicastGroupPort=${multicastGroupPort} can be replaced with the
-System property either from an environment variable or a system property specified with a
-command line switch such as -DmulticastGroupPort=4446. Another example, useful for Terracotta
-server based deployments is <terracottaConfig url="${serverAndPort}"/ and specify a command line
-switch of -Dserver36:9510
-
-The attributes of <ehcache> are:
-* name - an optional name for the CacheManager.  The name is optional and primarily used
-for documentation or to distinguish Terracotta clustered cache state.  With Terracotta
-clustered caches, a combination of CacheManager name and cache name uniquely identify a
-particular cache store in the Terracotta clustered memory.
-* updateCheck - an optional boolean flag specifying whether this CacheManager should check
-for new versions of Ehcache over the Internet.  If not specified, updateCheck="true".
-* dynamicConfig - an optional setting that can be used to disable dynamic configuration of caches
-associated with this CacheManager.  By default this is set to true - i.e. dynamic configuration
-is enabled.  Dynamically configurable caches can have their TTI, TTL and maximum disk and
-in-memory capacity changed at runtime through the cache's configuration object.
-* monitoring - an optional setting that determines whether the CacheManager should
-automatically register the SampledCacheMBean with the system MBean server.
-
-Currently, this monitoring is only useful when using Terracotta clustering and using the
-Terracotta Developer Console. With the "autodetect" value, the presence of Terracotta clustering
-will be detected and monitoring, via the Developer Console, will be enabled. Other allowed values
-are "on" and "off".  The default is "autodetect". This setting does not perform any function when
-used with JMX monitors.
-
-* maxBytesLocalHeap - optional setting that constraints the memory usage of the Caches managed by the CacheManager
-to use at most the specified number of bytes of the local VM's heap.
-* maxBytesLocalOffHeap - optional setting that constraints the offHeap usage of the Caches managed by the CacheManager
-to use at most the specified number of bytes of the local VM's offHeap memory.
-* maxBytesLocalDisk - optional setting that constraints the disk usage of the Caches managed by the CacheManager
-to use at most the specified number of bytes of the local disk.
-
-These settings let you define "resource pools", caches will share. For instance setting maxBytesLocalHeap to 100M, will result in
-all caches sharing 100 MegaBytes of ram. The CacheManager will balance these 100 MB across all caches based on their respective usage
-patterns. You can allocate a precise amount of bytes to a particular cache by setting the appropriate maxBytes* attribute for that cache.
-That amount will be subtracted from the CacheManager pools, so that if a cache a specified 30M requirement, the other caches will share
-the remaining 70M.
-
-Also, specifying a maxBytesLocalOffHeap at the CacheManager level will result in overflowToOffHeap to be true by default. If you don't want
-a specific cache to overflow to off heap, you'll have to set overflowToOffHeap="false" explicitly
-
-Here is an example of CacheManager level resource tuning, which will use up to 400M of heap and 2G of offHeap:
-
-<ehcache xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:noNamespaceSchemaLocation="ehcache.xsd"
-         updateCheck="true" monitoring="autodetect"
-         dynamicConfig="true" maxBytesLocalHeap="400M" maxBytesLocalOffHeap="2G">
-
--->
-<ehcache name="kiwi"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         updateCheck="false"
-         maxBytesLocalHeap="60%"
-         dynamicConfig="false"
-         xsi:noNamespaceSchemaLocation="http://ehcache.sf.net/ehcache.xsd">
-
-    <!-- 
-    DiskStore configuration
-    =======================
-
-    The diskStore element is optional. To turn off disk store path creation, comment out the diskStore
-    element below.
-
-    Configure it if you have overflowToDisk or diskPersistent enabled for any cache.
-
-    If it is not configured, and a cache is created which requires a disk store, a warning will be
-     issued and java.io.tmpdir will automatically be used.
-
-    diskStore has only one attribute - "path". It is the path to the directory where
-    .data and .index files will be created.
-
-    If the path is one of the following Java System Property it is replaced by its value in the
-    running VM. For backward compatibility these are not specified without being enclosed in the ${token}
-    replacement syntax.
-
-    The following properties are translated:
-    * user.home - KiWiUser's home directory
-    * user.dir - KiWiUser's current working directory
-    * java.io.tmpdir - Default temp file path
-    * ehcache.disk.store.dir - A system property you would normally specify on the command line
-      e.g. java -Dehcache.disk.store.dir=/u01/myapp/diskdir ...
-
-    Subdirectories can be specified below the property e.g. java.io.tmpdir/one
-
-    -->
-    <diskStore path="java.io.tmpdir/kiwi-cache"/>
-
-
-    <!--
-        Cache configuration
-        ===================
-
-        The following attributes are required.
-
-        name:
-        Sets the name of the cache. This is used to identify the cache. It must be unique.
-
-        maxEntriesLocalHeap:
-        Sets the maximum number of objects that will be held on heap memory.  0 = no limit.
-
-        maxEntriesLocalDisk:
-        Sets the maximum number of objects that will be maintained in the DiskStore
-        The default value is zero, meaning unlimited.
-
-        eternal:
-        Sets whether elements are eternal. If eternal,  timeouts are ignored and the
-        element is never expired.
-
-        The following attributes and elements are optional.
-
-        maxEntriesInCache:
-        This feature is applicable only to Terracotta distributed caches.
-        Sets the maximum number of entries that can be stored in the cluster. 0 = no limit.
-        Note that clustered cache will still perform eviction if resource usage requires it.
-        This property can be modified dynamically while the cache is operating.
-
-        overflowToOffHeap:
-        (boolean) This feature is available only in enterprise versions of Ehcache.
-        When set to true, enables the cache to utilize off-heap memory
-        storage to improve performance. Off-heap memory is not subject to Java
-        GC. The default value is false.
-
-        maxBytesLocalHeap:
-        Defines how many bytes the cache may use from the VM's heap. If a CacheManager
-        maxBytesLocalHeap has been defined, this Cache's specified amount will be
-        subtracted from the CacheManager. Other caches will share the remainder.
-        This attribute's values are given as <number>k|K|m|M|g|G for
-        kilobytes (k|K), megabytes (m|M), or gigabytes (g|G).
-        For example, maxBytesLocalHeap="2g" allots 2 gigabytes of heap memory.
-        If you specify a maxBytesLocalHeap, you can't use the maxEntriesLocalHeap attribute.
-        maxEntriesLocalHeap can't be used if a CacheManager maxBytesLocalHeap is set.
-
-        Elements put into the cache will be measured in size using net.sf.ehcache.pool.sizeof.SizeOf
-        If you wish to ignore some part of the object graph, see net.sf.ehcache.pool.sizeof.annotations.IgnoreSizeOf
-
-        maxBytesLocalOffHeap:
-        This feature is available only in enterprise versions of Ehcache.
-        Sets the amount of off-heap memory this cache can use, and will reserve.
-
-        This setting will set overflowToOffHeap to true. Set explicitly to false to disable overflow behavior.
-
-        Note that it is recommended to set maxEntriesLocalHeap to at least 100 elements
-        when using an off-heap store, otherwise performance will be seriously degraded,
-        and a warning will be logged.
-
-        The minimum amount that can be allocated is 128MB. There is no maximum.
-
-        maxBytesLocalDisk:
-        As for maxBytesLocalHeap, but specifies the limit of disk storage this cache will ever use.
-
-        timeToIdleSeconds:
-        Sets the time to idle for an element before it expires.
-        i.e. The maximum amount of time between accesses before an element expires
-        Is only used if the element is not eternal.
-        Optional attribute. A value of 0 means that an Element can idle for infinity.
-        The default value is 0.
-
-        timeToLiveSeconds:
-        Sets the time to live for an element before it expires.
-        i.e. The maximum time between creation time and when an element expires.
-        Is only used if the element is not eternal.
-        Optional attribute. A value of 0 means that and Element can live for infinity.
-        The default value is 0.
-
-        diskExpiryThreadIntervalSeconds:
-        The number of seconds between runs of the disk expiry thread. The default value
-        is 120 seconds.
-
-        diskSpoolBufferSizeMB:
-        This is the size to allocate the DiskStore for a spool buffer. Writes are made
-        to this area and then asynchronously written to disk. The default size is 30MB.
-        Each spool buffer is used only by its cache. If you get OutOfMemory errors consider
-        lowering this value. To improve DiskStore performance consider increasing it. Trace level
-        logging in the DiskStore will show if put back ups are occurring.
-
-        clearOnFlush:
-        whether the MemoryStore should be cleared when flush() is called on the cache.
-        By default, this is true i.e. the MemoryStore is cleared.
-
-        memoryStoreEvictionPolicy:
-        Policy would be enforced upon reaching the maxEntriesLocalHeap limit. Default
-        policy is Least Recently Used (specified as LRU). Other policies available -
-        First In First Out (specified as FIFO) and Less Frequently Used
-        (specified as LFU)
-
-        copyOnRead:
-        Whether an Element is copied when being read from a cache.
-        By default this is false.
-
-        copyOnWrite:
-        Whether an Element is copied when being added to the cache.
-        By default this is false.
-
-        Cache persistence is configured through the persistence sub-element.  The attributes of the
-        persistence element are:
-
-        strategy:
-        Configures the type of persistence provided by the configured cache.  This must be one of the
-        following values:
-
-        * localRestartable - Enables the RestartStore and copies all cache entries (on-heap and/or off-heap)
-        to disk. This option provides fast restartability with fault tolerant cache persistence on disk.
-        It is available for Enterprise Ehcache users only.
-
-        * localTempSwap - Swaps cache entries (on-heap and/or off-heap) to disk when the cache is full.
-        "localTempSwap" is not persistent.
-
-        * none - Does not persist cache entries.
-
-        * distributed - Defers to the <terracotta> configuration for persistence settings. This option
-        is not applicable for standalone.
-
-        synchronousWrites:
-        When set to true write operations on the cache do not return until after the operations data has been
-        successfully flushed to the disk storage.  This option is only valid when used with the "localRestartable"
-        strategy, and defaults to false.
-
-        The following example configuration shows a cache configured for localTempSwap restartability.
-
-        <cache name="persistentCache" maxEntriesLocalHeap="1000">
-            <persistence strategy="localTempSwap"/>
-        </cache>
-
-        Cache elements can also contain sub elements which take the same format of a factory class
-        and properties. Defined sub-elements are:
-
-        * cacheEventListenerFactory - Enables registration of listeners for cache events, such as
-          put, remove, update, and expire.
-
-        * bootstrapCacheLoaderFactory - Specifies a BootstrapCacheLoader, which is called by a
-          cache on initialisation to prepopulate itself.
-
-        * cacheExtensionFactory - Specifies a CacheExtension, a generic mechanism to tie a class
-          which holds a reference to a cache to the cache lifecycle.
-
-        * cacheExceptionHandlerFactory - Specifies a CacheExceptionHandler, which is called when
-          cache exceptions occur.
-
-        * cacheLoaderFactory - Specifies a CacheLoader, which can be used both asynchronously and
-          synchronously to load objects into a cache. More than one cacheLoaderFactory element
-          can be added, in which case the loaders form a chain which are executed in order. If a
-          loader returns null, the next in chain is called.
-
-        * copyStrategy - Specifies a fully qualified class which implements
-          net.sf.ehcache.store.compound.CopyStrategy. This strategy will be used for copyOnRead
-          and copyOnWrite in place of the default which is serialization.
-
-        Example of cache level resource tuning:
-        <cache name="memBound" maxBytesLocalHeap="100m" maxBytesLocalOffHeap="4g" maxBytesLocalDisk="200g" />
-
-
-        Cache Event Listeners
-        +++++++++++++++++++++
-
-        All cacheEventListenerFactory elements can take an optional property listenFor that describes
-        which events will be delivered in a clustered environment.  The listenFor attribute has the
-        following allowed values:
-
-        * all - the default is to deliver all local and remote events
-        * local - deliver only events originating in the current node
-        * remote - deliver only events originating in other nodes
-
-        Example of setting up a logging listener for local cache events:
-
-        <cacheEventListenerFactory class="my.company.log.CacheLogger"
-            listenFor="local" />
-
-
-        Search
-        ++++++
-
-        A <cache> can be made searchable by adding a <searchable/> sub-element. By default the keys
-        and value objects of elements put into the cache will be attributes against which
-        queries can be expressed.
-
-        <cache>
-            <searchable/>
-        </cache>
-
-
-        An "attribute" of the cache elements can also be defined to be searchable. In the example below
-        an attribute with the name "age" will be available for use in queries. The value for the "age"
-        attribute will be computed by calling the method "getAge()" on the value object of each element
-        in the cache. See net.sf.ehcache.search.attribute.ReflectionAttributeExtractor for the format of
-        attribute expressions. Attribute values must also conform to the set of types documented in the
-        net.sf.ehcache.search.attribute.AttributeExtractor interface
-
-        <cache>
-            <searchable>
-                <searchAttribute name="age" expression="value.getAge()"/>
-            </searchable>
-        </cache>
-
-
-        Attributes may also be defined using a JavaBean style. With the following attribute declaration
-        a public method getAge() will be expected to be found on either the key or value for cache elements
-
-        <cache>
-            <searchable>
-                <searchAttribute name="age"/>
-            </searchable>
-        </cache>
-
-        In more complex situations you can create your own attribute extractor by implementing the
-        AttributeExtractor interface. Providing your extractor class is shown in the following example:
-
-        <cache>
-            <searchable>
-                <searchAttribute name="age" class="com.example.MyAttributeExtractor"/>
-            </searchable>
-        </cache>
-
-        Use properties to pass state to your attribute extractor if needed. Your implementation must provide
-        a public constructor that takes a single java.util.Properties instance
-
-        <cache>
-            <searchable>
-                <searchAttribute name="age" class="com.example.MyAttributeExtractor" properties="foo=1,bar=2"/>
-            </searchable>
-        </cache>
-
-
-        RMI Cache Replication
-        +++++++++++++++++++++
-
-        Each cache that will be distributed needs to set a cache event listener which replicates
-        messages to the other CacheManager peers. For the built-in RMI implementation this is done
-        by adding a cacheEventListenerFactory element of type RMICacheReplicatorFactory to each
-        distributed cache's configuration as per the following example:
-
-        <cacheEventListenerFactory class="net.sf.ehcache.distribution.RMICacheReplicatorFactory"
-             properties="replicateAsynchronously=true,
-             replicatePuts=true,
-             replicatePutsViaCopy=false,
-             replicateUpdates=true,
-             replicateUpdatesViaCopy=true,
-             replicateRemovals=true,
-             asynchronousReplicationIntervalMillis=<number of milliseconds>,
-             asynchronousReplicationMaximumBatchSize=<number of operations>"
-             propertySeparator="," />
-
-        The RMICacheReplicatorFactory recognises the following properties:
-
-        * replicatePuts=true|false - whether new elements placed in a cache are
-          replicated to others. Defaults to true.
-
-        * replicatePutsViaCopy=true|false - whether the new elements are
-          copied to other caches (true), or whether a remove message is sent. Defaults to true.
-
-        * replicateUpdates=true|false - whether new elements which override an
-          element already existing with the same key are replicated. Defaults to true.
-
-        * replicateRemovals=true - whether element removals are replicated. Defaults to true.
-
-        * replicateAsynchronously=true | false - whether replications are
-          asynchronous (true) or synchronous (false). Defaults to true.
-
-        * replicateUpdatesViaCopy=true | false - whether the new elements are
-          copied to other caches (true), or whether a remove message is sent. Defaults to true.
-
-        * asynchronousReplicationIntervalMillis=<number of milliseconds> - The asynchronous
-          replicator runs at a set interval of milliseconds. The default is 1000. The minimum
-          is 10. This property is only applicable if replicateAsynchronously=true
-
-        * asynchronousReplicationMaximumBatchSize=<number of operations> - The maximum
-          number of operations that will be batch within a single RMI message.  The default
-          is 1000. This property is only applicable if replicateAsynchronously=true
-
-        JGroups Replication
-        +++++++++++++++++++
-
-        For the Jgroups replication this is done with:
-        <cacheEventListenerFactory class="net.sf.ehcache.distribution.jgroups.JGroupsCacheReplicatorFactory"
-                                properties="replicateAsynchronously=true, replicatePuts=true,
-                   replicateUpdates=true, replicateUpdatesViaCopy=false,
-                   replicateRemovals=true,asynchronousReplicationIntervalMillis=1000"/>
-        This listener supports the same properties as the RMICacheReplicationFactory.
-
-
-        JMS Replication
-        +++++++++++++++
-
-        For JMS-based replication this is done with:
-        <cacheEventListenerFactory
-              class="net.sf.ehcache.distribution.jms.JMSCacheReplicatorFactory"
-              properties="replicateAsynchronously=true,
-                           replicatePuts=true,
-                           replicateUpdates=true,
-                           replicateUpdatesViaCopy=true,
-                           replicateRemovals=true,
-                           asynchronousReplicationIntervalMillis=1000"
-               propertySeparator=","/>
-
-        This listener supports the same properties as the RMICacheReplicationFactory.
-
-        Cluster Bootstrapping
-        +++++++++++++++++++++
-
-        Bootstrapping a cluster may use a different mechanism to replication. e.g you can mix
-        JMS replication with bootstrap via RMI - just make sure you have the cacheManagerPeerProviderFactory
-        and cacheManagerPeerListenerFactory configured.
-
-        There are two bootstrapping mechanisms: RMI and JGroups.
-
-        RMI Bootstrap
-
-        The RMIBootstrapCacheLoader bootstraps caches in clusters where RMICacheReplicators are
-        used. It is configured as per the following example:
-
-        <bootstrapCacheLoaderFactory
-            class="net.sf.ehcache.distribution.RMIBootstrapCacheLoaderFactory"
-            properties="bootstrapAsynchronously=true, maximumChunkSizeBytes=5000000"
-            propertySeparator="," />
-
-        The RMIBootstrapCacheLoaderFactory recognises the following optional properties:
-
-        * bootstrapAsynchronously=true|false - whether the bootstrap happens in the background
-          after the cache has started. If false, bootstrapping must complete before the cache is
-          made available. The default value is true.
-
-        * maximumChunkSizeBytes=<integer> - Caches can potentially be very large, larger than the
-          memory limits of the VM. This property allows the bootstraper to fetched elements in
-          chunks. The default chunk size is 5000000 (5MB).
-
-        JGroups Bootstrap
-
-        Here is an example of bootstrap configuration using JGroups boostrap:
-
-        <bootstrapCacheLoaderFactory class="net.sf.ehcache.distribution.jgroups.JGroupsBootstrapCacheLoaderFactory"
-                                        properties="bootstrapAsynchronously=true"/>
-
-        The configuration properties are the same as for RMI above. Note that JGroups bootstrap only supports
-        asynchronous bootstrap mode.
-
-
-        Cache Exception Handling
-        ++++++++++++++++++++++++
-
-        By default, most cache operations will propagate a runtime CacheException on failure. An
-        interceptor, using a dynamic proxy, may be configured so that a CacheExceptionHandler can
-        be configured to intercept Exceptions. Errors are not intercepted.
-
-        It is configured as per the following example:
-
-          <cacheExceptionHandlerFactory class="com.example.ExampleExceptionHandlerFactory"
-                                          properties="logLevel=FINE"/>
-
-        Caches with ExceptionHandling configured are not of type Cache, but are of type Ehcache only,
-        and are not available using CacheManager.getCache(), but using CacheManager.getEhcache().
-
-
-        Cache Loader
-        ++++++++++++
-
-        A default CacheLoader may be set which loads objects into the cache through asynchronous and
-        synchronous methods on Cache. This is different to the bootstrap cache loader, which is used
-        only in distributed caching.
-
-        It is configured as per the following example:
-
-            <cacheLoaderFactory class="com.example.ExampleCacheLoaderFactory"
-                                          properties="type=int,startCounter=10"/>
-
-        Element value comparator
-        ++++++++++++++++++++++++
-
-        These two cache atomic methods:
-          removeElement(Element e)
-          replace(Element old, Element element)
-
-        rely on comparison of cached elements value. The default implementation relies on Object.equals()
-        but that can be changed in case you want to use a different way to compute equality of two elements.
-
-        This is configured as per the following example:
-
-        <elementValueComparator class="com.company.xyz.MyElementComparator"/>
-
-        The MyElementComparator class must implement the is net.sf.ehcache.store.ElementValueComparator
-        interface. The default implementation is net.sf.ehcache.store.DefaultElementValueComparator.
-
-
-        SizeOf Policy
-        +++++++++++++
-
-        Control how deep the SizeOf engine can go when sizing on-heap elements.
-
-        This is configured as per the following example:
-
-        <sizeOfPolicy maxDepth="100" maxDepthExceededBehavior="abort"/>
-
-        maxDepth controls how many linked objects can be visited before the SizeOf engine takes any action.
-        maxDepthExceededBehavior specifies what happens when the max depth is exceeded while sizing an object graph.
-         "continue" makes the SizeOf engine log a warning and continue the sizing. This is the default.
-         "abort"    makes the SizeOf engine abort the sizing, log a warning and mark the cache as not correctly tracking
-                    memory usage. This makes Ehcache.hasAbortedSizeOf() return true when this happens.
-
-        The SizeOf policy can be configured at the cache manager level (directly under <ehcache>) and at
-        the cache level (under <cache> or <defaultCache>). The cache policy always overrides the cache manager
-        one if both are set. This element has no effect on distributed caches.
-
-        Transactions
-        ++++++++++++
-
-        To enable an ehcache as transactions, set the transactionalMode
-
-        transactionalMode="xa" - high performance JTA/XA implementation
-        transactionalMode="xa_strict" - canonically correct JTA/XA implementation
-        transactionMode="local" - high performance local transactions involving caches only
-        transactionalMode="off" - the default, no transactions
-
-        If set, all cache operations will need to be done through transactions.
-
-        To prevent users keeping references on stored elements and modifying them outside of any transaction's control,
-        transactions also require the cache to be configured copyOnRead and copyOnWrite.
-
-        CacheWriter
-        ++++++++++++
-
-        A CacheWriter can be set to write to an underlying resource. Only one CacheWriter can be
-        configured per cache.
-
-        The following is an example of how to configure CacheWriter for write-through:
-
-            <cacheWriter writeMode="write-through" notifyListenersOnException="true">
-                <cacheWriterFactory class="net.sf.ehcache.writer.TestCacheWriterFactory"
-                                    properties="type=int,startCounter=10"/>
-            </cacheWriter>
-
-        The following is an example of how to configure CacheWriter for write-behind:
-
-            <cacheWriter writeMode="write-behind" minWriteDelay="1" maxWriteDelay="5"
-                         rateLimitPerSecond="5" writeCoalescing="true" writeBatching="true" writeBatchSize="1"
-                         retryAttempts="2" retryAttemptDelaySeconds="1">
-                <cacheWriterFactory class="net.sf.ehcache.writer.TestCacheWriterFactory"
-                                    properties="type=int,startCounter=10"/>
-            </cacheWriter>
-
-        The cacheWriter element has the following attributes:
-        * writeMode: the write mode, write-through or write-behind
-
-        These attributes only apply to write-through mode:
-        * notifyListenersOnException: Sets whether to notify listeners when an exception occurs on a writer operation.
-
-        These attributes only apply to write-behind mode:
-        * minWriteDelay: Set the minimum number of seconds to wait before writing behind. If set to a value greater than 0,
-          it permits operations to build up in the queue. This is different from the maximum write delay in that by waiting
-          a minimum amount of time, work is always being built up. If the minimum write delay is set to zero and the
-          CacheWriter performs its work very quickly, the overhead of processing the write behind queue items becomes very
-          noticeable in a cluster since all the operations might be done for individual items instead of for a collection
-          of them.
-        * maxWriteDelay: Set the maximum number of seconds to wait before writing behind. If set to a value greater than 0,
-          it permits operations to build up in the queue to enable effective coalescing and batching optimisations.
-        * writeBatching: Sets whether to batch write operations. If set to true, writeAll and deleteAll will be called on
-          the CacheWriter rather than write and delete being called for each key. Resources such as databases can perform
-          more efficiently if updates are batched, thus reducing load.
-        * writeBatchSize: Sets the number of operations to include in each batch when writeBatching is enabled. If there are
-          less entries in the write-behind queue than the batch size, the queue length size is used.
-        * rateLimitPerSecond: Sets the maximum number of write operations to allow per second when writeBatching is enabled.
-        * writeCoalescing: Sets whether to use write coalescing. If set to true and multiple operations on the same key are
-          present in the write-behind queue, only the latest write is done, as the others are redundant.
-        * retryAttempts: Sets the number of times the operation is retried in the CacheWriter, this happens after the
-          original operation.
-        * retryAttemptDelaySeconds: Sets the number of seconds to wait before retrying an failed operation.
-
-        Cache Extension
-        +++++++++++++++
-
-        CacheExtensions are a general purpose mechanism to allow generic extensions to a Cache.
-        CacheExtensions are tied into the Cache lifecycle.
-
-        CacheExtensions are created using the CacheExtensionFactory which has a
-        <code>createCacheCacheExtension()</code> method which takes as a parameter a
-        Cache and properties. It can thus call back into any public method on Cache, including, of
-        course, the load methods.
-
-        Extensions are added as per the following example:
-
-             <cacheExtensionFactory class="com.example.FileWatchingCacheRefresherExtensionFactory"
-                                 properties="refreshIntervalMillis=18000, loaderTimeout=3000,
-                                             flushPeriod=whatever, someOtherProperty=someValue ..."/>
-
-        Cache Decorator Factory
-        +++++++++++++++++++++++
-
-        Cache decorators can be configured directly in ehcache.xml. The decorators will be created and added to the CacheManager.
-        It accepts the name of a concrete class that extends net.sf.ehcache.constructs.CacheDecoratorFactory
-        The properties will be parsed according to the delimiter (default is comma ',') and passed to the concrete factory's
-        <code>createDecoratedEhcache(Ehcache cache, Properties properties)</code> method along with the reference to the owning cache.
-
-        It is configured as per the following example:
-
-            <cacheDecoratorFactory
-          class="com.company.DecoratedCacheFactory"
-          properties="property1=true ..." />
-
-        Distributed Caching with Terracotta
-        +++++++++++++++++++++++++++++++++++
-
-        Distributed Caches connect to a Terracotta Server Array. They are configured with the <terracotta> sub-element.
-
-        The <terracotta> sub-element has the following attributes:
-
-        * clustered=true|false - indicates whether this cache should be clustered (distributed) with Terracotta. By
-          default, if the <terracotta> element is included, clustered=true.
-
-        * copyOnRead=true|false - indicates whether cache values are deserialized on every read or if the
-          materialized cache value can be re-used between get() calls. This setting is useful if a cache
-          is being shared by callers with disparate classloaders or to prevent local drift if keys/values
-          are mutated locally without being put back in the cache.
-
-          The default is false.
-
-        * consistency=strong|eventual - Indicates whether this cache should have strong consistency or eventual
-          consistency. The default is eventual. See the documentation for the meaning of these terms.
-
-        * synchronousWrites=true|false
-
-          Synchronous writes (synchronousWrites="true")  maximize data safety by blocking the client thread until
-          the write has been written to the Terracotta Server Array.
-
-          This option is only available with consistency=strong. The default is false.
-
-        * concurrency - the number of segments that will be used by the map underneath the Terracotta Store.
-          Its optional and has default value of 0, which means will use default values based on the internal
-          Map being used underneath the store.
-
-          This value cannot be changed programmatically once a cache is initialized.
-
-        The <terracotta> sub-element also has a <nonstop> sub-element to allow configuration of cache behaviour if a distributed
-        cache operation cannot be completed within a set time or in the event of a clusterOffline message. If this element does not appear, nonstop behavior is off.
-
-        <nonstop> has the following attributes:
-
-        *  enabled="true" - defaults to true.
-
-        *  timeoutMillis - An SLA setting, so that if a cache operation takes longer than the allowed ms, it will timeout.
-
-        *  searchTimeoutMillis - If a cache search operation in the nonstop mode takes longer than the allowed ms, it will timeout.
-
-        *  immediateTimeout="true|false" - What to do on receipt of a ClusterOffline event indicating that communications
-           with the Terracotta Server Array were interrupted.
-
-        <nonstop> has one sub-element, <timeoutBehavior> which has the following attribute:
-
-        *  type="noop|exception|localReads|localReadsAndExceptionOnWrite" - What to do when a timeout has occurred. Exception is the default.
-
-        Simplest example to indicate clustering:
-            <terracotta/>
-
-        To indicate the cache should not be clustered (or remove the <terracotta> element altogether):
-            <terracotta clustered="false"/>
-
-        To indicate the cache should be clustered using "eventual" consistency mode for better performance :
-            <terracotta clustered="true" consistency="eventual"/>
-
-        To indicate the cache should be clustered using synchronous-write locking level:
-            <terracotta clustered="true" synchronousWrites="true"/>
-        -->
-
-    <!--
-    Mandatory Default Cache configuration. These settings will be applied to caches
-    created programmtically using CacheManager.add(String cacheName)
-    -->
-    <defaultCache
-            eternal="true"
-            overflowToDisk="false"
-            memoryStoreEvictionPolicy="LRU"
-            />
-
-    <!--
-       a cache from database ID to KiWiNode; should be very large since this kind of lookup is a very frequent operation
-    -->
-    <cache name="node-cache"
-           maxBytesLocalHeap="30%"
-           timeToLiveSeconds="3600"
-           overflowToDisk="false"
-           memoryStoreEvictionPolicy="LFU"/>
-
-    <!--
-        a cache from database ID to KiWiTriple; should be fairly large since it can speed up loading query results
-        from the database by avoiding reconstructing each triple from the database result
-    -->
-    <cache name="triple-cache"
-           maxBytesLocalHeap="10%"
-           overflowToDisk="false"
-           timeToLiveSeconds="3600"
-           memoryStoreEvictionPolicy="LFU"/>
-
-
-    <!-- a cache from URI to KiWiUriResource -->
-    <cache name="uri-cache"
-           maxBytesLocalHeap="10%"
-           overflowToDisk="false"
-           timeToLiveSeconds="3600"
-           memoryStoreEvictionPolicy="LRU"/>
-
-    <!-- a cache from anonymous ID to KiWiAnonResource -->
-    <cache name="bnode-cache"
-           maxBytesLocalHeap="5%"
-           overflowToDisk="false"
-           timeToLiveSeconds="3600"
-           memoryStoreEvictionPolicy="LRU"/>
-
-
-    <!-- a cache from literal cache key to KiWiLiteral -->
-    <cache name="literal-cache"
-           maxBytesLocalHeap="10%"
-           overflowToDisk="false"
-           timeToLiveSeconds="3600"
-           memoryStoreEvictionPolicy="LFU"/>
-
-
-    <cache name="namespace-prefix-cache"
-           maxBytesLocalHeap="5%"
-           eternal="true"
-           overflowToDisk="false"
-           memoryStoreEvictionPolicy="LFU"/>
-
-    <cache name="namespace-uri-cache"
-           maxBytesLocalHeap="5%"
-           eternal="true"
-           overflowToDisk="false"
-           memoryStoreEvictionPolicy="LFU"/>
-
-
-    <cache name="loader-cache"
-           statistics="true"
-           maxBytesLocalHeap="15%"
-           timeToIdleSeconds="120"
-           overflowToDisk="false"
-           memoryStoreEvictionPolicy="LRU"/>
-
-
-    <!--  uncomment to enable cache debugging -->
-<!-- 
-	<cacheManagerPeerListenerFactory
-	    class="org.terracotta.ehcachedx.monitor.probe.ProbePeerListenerFactory"
-	    properties="monitorAddress=localhost, monitorPort=9889" />
--->
-
-</ehcache>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/marmotta/blob/0d79835c/libraries/kiwi/kiwi-triplestore/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/libraries/kiwi/kiwi-triplestore/src/test/resources/logback.xml b/libraries/kiwi/kiwi-triplestore/src/test/resources/logback.xml
index 9678d31..42350aa 100644
--- a/libraries/kiwi/kiwi-triplestore/src/test/resources/logback.xml
+++ b/libraries/kiwi/kiwi-triplestore/src/test/resources/logback.xml
@@ -22,7 +22,7 @@
         </encoder>
     </appender>
 
-    <logger name="net.sf.ehcache.pool.impl" level="WARN" />
+    <logger name="org.infinispan.jmx" level="WARN" />
 
     <root level="${root-level:-INFO}">
         <appender-ref ref="CONSOLE"/>