You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@unomi.apache.org by jk...@apache.org on 2023/02/03 15:03:12 UTC

[unomi] branch master updated: UNOMI-724: Implem for rollover system to replace monthly… (#567)

This is an automated email from the ASF dual-hosted git repository.

jkevan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/unomi.git


The following commit(s) were added to refs/heads/master by this push:
     new 7feff5c50 UNOMI-724: Implem for rollover system to replace monthly… (#567)
7feff5c50 is described below

commit 7feff5c5067ff679a079211d9f2eb989898efc75
Author: kevan Jahanshahi <jk...@apache.org>
AuthorDate: Fri Feb 3 16:03:07 2023 +0100

    UNOMI-724: Implem for rollover system to replace monthly… (#567)
    
    * UNOMI-724: base POC and implem for rollover system to replace monthly indices
    
    * UNOMI-724 :
    - clean up POC
    - deprecate dateHint usage
    - add "rollover" properties to replace deprecated "monthlyIndex"
    - disable failing purge integration test
    
    * cleanup a bit + add support of prefix for the ILM policy registering to avoid potential conflicts
    
    * more cleanup
    
    * more cleanup
    
    ---------
    
    Co-authored-by: David Griffon <dg...@jahia.com>
---
 .../apache/unomi/api/services/ProfileService.java  |  12 +
 .../apache/unomi/services/UserListServiceImpl.java |   8 +-
 .../unomi/privacy/internal/PrivacyServiceImpl.java |   6 +-
 .../org/apache/unomi/itests/ContextServletIT.java  |   3 +-
 .../org/apache/unomi/itests/ProfileServiceIT.java  |   2 +
 .../main/resources/etc/custom.system.properties    |  14 +
 .../ElasticSearchPersistenceServiceImpl.java       | 365 +++++++++++++--------
 .../resources/OSGI-INF/blueprint/blueprint.xml     |  18 +
 .../org.apache.unomi.persistence.elasticsearch.cfg |  16 +
 .../ElasticsearchPersistenceTest.java              |   6 +-
 .../unomi/persistence/spi/PersistenceService.java  | 262 +++++++++------
 .../actions/MergeProfilesOnPropertyAction.java     |   8 +-
 .../unomi/plugins/mail/actions/SendMailAction.java |   2 +-
 .../rest/endpoints/ProfileServiceEndPoint.java     |   7 +-
 .../rest/service/impl/RestServiceUtilsImpl.java    |   2 +-
 .../services/impl/profiles/ProfileServiceImpl.java |  23 +-
 .../services/impl/segments/SegmentServiceImpl.java |   8 +-
 .../apache/unomi/shell/commands/SessionView.java   |   2 +-
 18 files changed, 491 insertions(+), 273 deletions(-)

diff --git a/api/src/main/java/org/apache/unomi/api/services/ProfileService.java b/api/src/main/java/org/apache/unomi/api/services/ProfileService.java
index 86fac9f61..527e77fec 100644
--- a/api/src/main/java/org/apache/unomi/api/services/ProfileService.java
+++ b/api/src/main/java/org/apache/unomi/api/services/ProfileService.java
@@ -175,13 +175,25 @@ public interface ProfileService {
 
     /**
      * Retrieves the session identified by the specified identifier.
+     * @deprecated {@code dateHint} is not supported anymore, please use {@link #loadSession(String)}
      *
      * @param sessionId the identifier of the session to be retrieved
      * @param dateHint  a Date helping in identifying where the item is located
      * @return the session identified by the specified identifier
      */
+    @Deprecated
     Session loadSession(String sessionId, Date dateHint);
 
+    /**
+     * Retrieves the session identified by the specified identifier.
+     *
+     * @param sessionId the identifier of the session to be retrieved
+     * @return the session identified by the specified identifier
+     */
+    default Session loadSession(String sessionId) {
+        return loadSession(sessionId, null);
+    };
+
     /**
      * Saves the specified session.
      *
diff --git a/extensions/lists-extension/services/src/main/java/org/apache/unomi/services/UserListServiceImpl.java b/extensions/lists-extension/services/src/main/java/org/apache/unomi/services/UserListServiceImpl.java
index 37ca72e04..ba1e74b9a 100644
--- a/extensions/lists-extension/services/src/main/java/org/apache/unomi/services/UserListServiceImpl.java
+++ b/extensions/lists-extension/services/src/main/java/org/apache/unomi/services/UserListServiceImpl.java
@@ -56,7 +56,7 @@ public class UserListServiceImpl implements UserListService {
     }
 
     public PartialList<Metadata> getListMetadatas(Query query) {
-        if(query.isForceRefresh()){
+        if (query.isForceRefresh()) {
             persistenceService.refresh();
         }
         definitionsService.resolveConditionType(query.getCondition());
@@ -89,12 +89,12 @@ public class UserListServiceImpl implements UserListService {
         Map<String, Object> profileSystemProperties;
         for (Profile p : profiles) {
             profileSystemProperties = p.getSystemProperties();
-            if(profileSystemProperties != null && profileSystemProperties.get("lists") != null) {
+            if (profileSystemProperties != null && profileSystemProperties.get("lists") != null) {
                 int index = ((List) profileSystemProperties.get("lists")).indexOf(listId);
-                if(index != -1){
+                if (index != -1) {
                     ((List) profileSystemProperties.get("lists")).remove(index);
                     profileSystemProperties.put("lastUpdated", new Date());
-                    persistenceService.update(p, null, Profile.class, "systemProperties", profileSystemProperties);
+                    persistenceService.update(p, Profile.class, "systemProperties", profileSystemProperties);
                 }
             }
         }
diff --git a/extensions/privacy-extension/services/src/main/java/org/apache/unomi/privacy/internal/PrivacyServiceImpl.java b/extensions/privacy-extension/services/src/main/java/org/apache/unomi/privacy/internal/PrivacyServiceImpl.java
index 9c59f3d2c..fb357d559 100644
--- a/extensions/privacy-extension/services/src/main/java/org/apache/unomi/privacy/internal/PrivacyServiceImpl.java
+++ b/extensions/privacy-extension/services/src/main/java/org/apache/unomi/privacy/internal/PrivacyServiceImpl.java
@@ -99,7 +99,7 @@ public class PrivacyServiceImpl implements PrivacyService {
         if (profile == null) {
             return false;
         }
-        eventService.send(new Event("profileDeleted", null, profile, null, null, profile,null, new Date(), false));
+        eventService.send(new Event("profileDeleted", null, profile, null, null, profile, null, new Date(), false));
         // we simply overwrite the existing profile with an empty one.
         Profile emptyProfile = new Profile(profileId);
         profileService.save(emptyProfile);
@@ -142,7 +142,7 @@ public class PrivacyServiceImpl implements PrivacyService {
             persistenceService.save(session);
             List<Event> events = eventService.searchEvents(session.getItemId(), new String[0], null, 0, -1, null).getList();
             for (Event event : events) {
-                persistenceService.update(event, event.getTimeStamp(), Event.class, "profileId", newProfile.getItemId());
+                persistenceService.update(event, Event.class, "profileId", newProfile.getItemId());
             }
         }
 
@@ -150,7 +150,7 @@ public class PrivacyServiceImpl implements PrivacyService {
     }
 
     @Override
-    public Boolean deleteProfileData(String profileId,boolean purgeData) {
+    public Boolean deleteProfileData(String profileId, boolean purgeData) {
         if (purgeData) {
             eventService.removeProfileEvents(profileId);
             profileService.removeProfileSessions(profileId);
diff --git a/itests/src/test/java/org/apache/unomi/itests/ContextServletIT.java b/itests/src/test/java/org/apache/unomi/itests/ContextServletIT.java
index c27a926d7..cd40c44b0 100644
--- a/itests/src/test/java/org/apache/unomi/itests/ContextServletIT.java
+++ b/itests/src/test/java/org/apache/unomi/itests/ContextServletIT.java
@@ -199,8 +199,7 @@ public class ContextServletIT extends BaseIT {
         TestUtils.executeContextJSONRequest(request, sessionId);
 
         Session session = keepTrying("Session with the id " + sessionId + " not saved in the required time",
-                () -> profileService.loadSession(sessionId,
-                        null), Objects::nonNull, DEFAULT_TRYING_TIMEOUT,
+                () -> profileService.loadSession(sessionId), Objects::nonNull, DEFAULT_TRYING_TIMEOUT,
                 DEFAULT_TRYING_TRIES);
 
         assertEquals(TEST_EVENT_TYPE, session.getOriginEventTypes().get(0));
diff --git a/itests/src/test/java/org/apache/unomi/itests/ProfileServiceIT.java b/itests/src/test/java/org/apache/unomi/itests/ProfileServiceIT.java
index 30bf8dcf1..79a3bf843 100644
--- a/itests/src/test/java/org/apache/unomi/itests/ProfileServiceIT.java
+++ b/itests/src/test/java/org/apache/unomi/itests/ProfileServiceIT.java
@@ -24,6 +24,7 @@ import org.apache.unomi.persistence.spi.PersistenceService;
 import org.apache.unomi.schema.api.JsonSchemaWrapper;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.ops4j.pax.exam.junit.PaxExam;
@@ -330,6 +331,7 @@ public class ProfileServiceIT extends BaseIT {
     }
 
     @Test
+    @Ignore // TODO - fix test  https://issues.apache.org/jira/browse/UNOMI-726
     public void testMonthlyIndicesPurge() throws Exception {
         Date currentDate = new Date();
         LocalDateTime minus10Months = LocalDateTime.ofInstant(currentDate.toInstant(), ZoneId.systemDefault()).minusMonths(10);
diff --git a/package/src/main/resources/etc/custom.system.properties b/package/src/main/resources/etc/custom.system.properties
index d0525ca5d..e8b049d40 100644
--- a/package/src/main/resources/etc/custom.system.properties
+++ b/package/src/main/resources/etc/custom.system.properties
@@ -106,11 +106,25 @@ org.apache.unomi.elasticsearch.addresses=${env:UNOMI_ELASTICSEARCH_ADDRESSES:-lo
 org.apache.unomi.elasticsearch.itemTypeToRefreshPolicy=${env:UNOMI_ELASTICSEARCH_REFRESH_POLICY_PER_ITEM_TYPE:-}
 org.apache.unomi.elasticsearch.fatalIllegalStateErrors=${env:UNOMI_ELASTICSEARCH_FATAL_STATE_ERRORS:-}
 org.apache.unomi.elasticsearch.index.prefix=${env:UNOMI_ELASTICSEARCH_INDEXPREFIX:-context}
+
+# These monthlyIndex properties are now deprecated, please use rollover equivalent.
 org.apache.unomi.elasticsearch.monthlyIndex.nbShards=${env:UNOMI_ELASTICSEARCH_MONTHLYINDEX_SHARDS:-5}
 org.apache.unomi.elasticsearch.monthlyIndex.nbReplicas=${env:UNOMI_ELASTICSEARCH_MONTHLYINDEX_REPLICAS:-0}
 org.apache.unomi.elasticsearch.monthlyIndex.indexMappingTotalFieldsLimit=${env:UNOMI_ELASTICSEARCH_MONTHLYINDEX_MAPPINGTOTALFIELDSLIMIT:-1000}
 org.apache.unomi.elasticsearch.monthlyIndex.indexMaxDocValueFieldsSearch=${env:UNOMI_ELASTICSEARCH_MONTHLYINDEX_MAXDOCVALUEFIELDSSEARCH:-1000}
 org.apache.unomi.elasticsearch.monthlyIndex.itemsMonthlyIndexedOverride=${env:UNOMI_ELASTICSEARCH_MONTHLYINDEX_ITEMSMONTHLYINDEXED:-event,session}
+# New rollover properties (it overrides monthlyIndex values)
+org.apache.unomi.elasticsearch.rollover.nbShards=${env:UNOMI_ELASTICSEARCH_ROLLOVER_SHARDS}
+org.apache.unomi.elasticsearch.rollover.nbReplicas=${env:UNOMI_ELASTICSEARCH_ROLLOVER_REPLICAS}
+org.apache.unomi.elasticsearch.rollover.indexMappingTotalFieldsLimit=${env:UNOMI_ELASTICSEARCH_ROLLOVER_MAPPINGTOTALFIELDSLIMIT}
+org.apache.unomi.elasticsearch.rollover.indexMaxDocValueFieldsSearch=${env:UNOMI_ELASTICSEARCH_ROLLOVER_MAXDOCVALUEFIELDSSEARCH}
+org.apache.unomi.elasticsearch.rollover.indices=${env:UNOMI_ELASTICSEARCH_ROLLOVER_INDICES}
+
+# Rollover configuration
+org.apache.unomi.elasticsearch.rollover.maxSize=${env:UNOMI_ELASTICSEARCH_ROLLOVER_MAXSIZE:-}
+org.apache.unomi.elasticsearch.rollover.maxAge=${env:UNOMI_ELASTICSEARCH_ROLLOVER_MAXAGE:-365d}
+org.apache.unomi.elasticsearch.rollover.maxDocs=${env:UNOMI_ELASTICSEARCH_ROLLOVER_MAXDOCS:-}
+
 org.apache.unomi.elasticsearch.defaultIndex.nbShards=${env:UNOMI_ELASTICSEARCH_DEFAULTINDEX_SHARDS:-5}
 org.apache.unomi.elasticsearch.defaultIndex.nbReplicas=${env:UNOMI_ELASTICSEARCH_DEFAULTINDEX_REPLICAS:-0}
 org.apache.unomi.elasticsearch.defaultIndex.indexMappingTotalFieldsLimit=${env:UNOMI_ELASTICSEARCH_DEFAULTINDEX_MAPPINGTOTALFIELDSLIMIT:-1000}
diff --git a/persistence-elasticsearch/core/src/main/java/org/apache/unomi/persistence/elasticsearch/ElasticSearchPersistenceServiceImpl.java b/persistence-elasticsearch/core/src/main/java/org/apache/unomi/persistence/elasticsearch/ElasticSearchPersistenceServiceImpl.java
index 63150feec..864548f3e 100644
--- a/persistence-elasticsearch/core/src/main/java/org/apache/unomi/persistence/elasticsearch/ElasticSearchPersistenceServiceImpl.java
+++ b/persistence-elasticsearch/core/src/main/java/org/apache/unomi/persistence/elasticsearch/ElasticSearchPersistenceServiceImpl.java
@@ -48,10 +48,11 @@ import org.apache.unomi.persistence.spi.aggregate.IpRangeAggregate;
 import org.apache.unomi.persistence.spi.aggregate.NumericRangeAggregate;
 import org.apache.unomi.persistence.spi.aggregate.TermsAggregate;
 import org.elasticsearch.ElasticsearchStatusException;
-import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.Version;
 import org.elasticsearch.action.DocWriteRequest;
 import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
 import org.elasticsearch.action.admin.cluster.storedscripts.PutStoredScriptRequest;
+import org.elasticsearch.action.admin.indices.alias.Alias;
 import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
 import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
 import org.elasticsearch.action.admin.indices.template.delete.DeleteIndexTemplateRequest;
@@ -83,6 +84,7 @@ import org.elasticsearch.action.update.UpdateResponse;
 import org.elasticsearch.client.core.CountRequest;
 import org.elasticsearch.client.core.CountResponse;
 import org.elasticsearch.client.core.MainResponse;
+import org.elasticsearch.client.indexlifecycle.*;
 import org.elasticsearch.client.indices.CreateIndexRequest;
 import org.elasticsearch.client.indices.CreateIndexResponse;
 import org.elasticsearch.client.indices.GetIndexRequest;
@@ -173,21 +175,18 @@ import static org.elasticsearch.index.query.QueryBuilders.termQuery;
 @SuppressWarnings("rawtypes")
 public class ElasticSearchPersistenceServiceImpl implements PersistenceService, SynchronousBundleListener {
 
-    public static final String NUMBER_OF_SHARDS = "number_of_shards";
-    public static final String NUMBER_OF_REPLICAS = "number_of_replicas";
-    public static final String CLUSTER_NAME = "cluster.name";
-    public static final String BULK_PROCESSOR_NAME = "bulkProcessor.name";
     public static final String BULK_PROCESSOR_CONCURRENT_REQUESTS = "bulkProcessor.concurrentRequests";
     public static final String BULK_PROCESSOR_BULK_ACTIONS = "bulkProcessor.bulkActions";
     public static final String BULK_PROCESSOR_BULK_SIZE = "bulkProcessor.bulkSize";
-    public static final String MONTHLY_INDEX_ITEMS_MONTHLY_INDEXED = "monthlyIndex.itemsMonthlyIndexedOverride";
     public static final String BULK_PROCESSOR_FLUSH_INTERVAL = "bulkProcessor.flushInterval";
     public static final String BULK_PROCESSOR_BACKOFF_POLICY = "bulkProcessor.backoffPolicy";
+    public static final String MONTHLY_INDEX_ITEMS_MONTHLY_INDEXED = "monthlyIndex.itemsMonthlyIndexedOverride";
     public static final String INDEX_DATE_PREFIX = "date-";
     public static final String SEQ_NO = "seq_no";
     public static final String PRIMARY_TERM = "primary_term";
 
     private static final Logger logger = LoggerFactory.getLogger(ElasticSearchPersistenceServiceImpl.class.getName());
+    private static final String ROLLOVER_LIFECYCLE_NAME = "unomi-rollover-policy";
     private boolean throwExceptions = false;
     private RestHighLevelClient client;
     private BulkProcessor bulkProcessor;
@@ -208,20 +207,28 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
     private Map<String, String> mappings = new HashMap<String, String>();
     private ConditionEvaluatorDispatcher conditionEvaluatorDispatcher;
     private ConditionESQueryBuilderDispatcher conditionESQueryBuilderDispatcher;
-
     private List<String> itemsMonthlyIndexed;
     private Map<String, String> routingByType;
 
     private Integer defaultQueryLimit = 10;
     private Integer removeByQueryTimeoutInMinutes = 10;
 
-    private String itemsMonthlyIndexedOverride = "event,session";
     private String bulkProcessorConcurrentRequests = "1";
     private String bulkProcessorBulkActions = "1000";
     private String bulkProcessorBulkSize = "5MB";
     private String bulkProcessorFlushInterval = "5s";
     private String bulkProcessorBackoffPolicy = "exponential";
 
+    // Rollover configuration
+    private List<String> rolloverIndices;
+    private String rolloverMaxSize;
+    private String rolloverMaxAge;
+    private String rolloverMaxDocs;
+    private String rolloverIndexNumberOfShards;
+    private String rolloverIndexNumberOfReplicas;
+    private String rolloverIndexMappingTotalFieldsLimit;
+    private String rolloverIndexMaxDocValueFieldsSearch;
+
     private String minimalElasticSearchVersion = "7.0.0";
     private String maximalElasticSearchVersion = "8.0.0";
 
@@ -265,7 +272,8 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
     public void setItemTypeToRefreshPolicy(String itemTypeToRefreshPolicy) throws IOException {
         if (!itemTypeToRefreshPolicy.isEmpty()) {
             this.itemTypeToRefreshPolicy = new ObjectMapper().readValue(itemTypeToRefreshPolicy,
-                        new TypeReference<HashMap<String, WriteRequest.RefreshPolicy>>() {});
+                    new TypeReference<HashMap<String, WriteRequest.RefreshPolicy>>() {
+                    });
         }
     }
 
@@ -284,22 +292,31 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         this.indexPrefix = indexPrefix;
     }
 
+    @Deprecated
     public void setMonthlyIndexNumberOfShards(String monthlyIndexNumberOfShards) {
         this.monthlyIndexNumberOfShards = monthlyIndexNumberOfShards;
     }
 
+    @Deprecated
     public void setMonthlyIndexNumberOfReplicas(String monthlyIndexNumberOfReplicas) {
         this.monthlyIndexNumberOfReplicas = monthlyIndexNumberOfReplicas;
     }
 
+    @Deprecated
     public void setMonthlyIndexMappingTotalFieldsLimit(String monthlyIndexMappingTotalFieldsLimit) {
         this.monthlyIndexMappingTotalFieldsLimit = monthlyIndexMappingTotalFieldsLimit;
     }
 
+    @Deprecated
     public void setMonthlyIndexMaxDocValueFieldsSearch(String monthlyIndexMaxDocValueFieldsSearch) {
         this.monthlyIndexMaxDocValueFieldsSearch = monthlyIndexMaxDocValueFieldsSearch;
     }
 
+    @Deprecated
+    public void setItemsMonthlyIndexedOverride(String itemsMonthlyIndexedOverride) {
+        this.itemsMonthlyIndexed = StringUtils.isNotEmpty(itemsMonthlyIndexedOverride) ? Arrays.asList(itemsMonthlyIndexedOverride.split(",").clone()) : Collections.emptyList();
+    }
+
     public void setNumberOfShards(String numberOfShards) {
         this.numberOfShards = numberOfShards;
     }
@@ -320,10 +337,6 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         this.defaultQueryLimit = defaultQueryLimit;
     }
 
-    public void setItemsMonthlyIndexedOverride(String itemsMonthlyIndexedOverride) {
-        this.itemsMonthlyIndexedOverride = itemsMonthlyIndexedOverride;
-    }
-
     public void setRoutingByType(Map<String, String> routingByType) {
         this.routingByType = routingByType;
     }
@@ -356,6 +369,38 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         this.bulkProcessorBackoffPolicy = bulkProcessorBackoffPolicy;
     }
 
+    public void setRolloverIndices(String rolloverIndices) {
+        this.rolloverIndices = StringUtils.isNotEmpty(rolloverIndices) ? Arrays.asList(rolloverIndices.split(",").clone()) : null;
+    }
+
+    public void setRolloverMaxSize(String rolloverMaxSize) {
+        this.rolloverMaxSize = rolloverMaxSize;
+    }
+
+    public void setRolloverMaxAge(String rolloverMaxAge) {
+        this.rolloverMaxAge = rolloverMaxAge;
+    }
+
+    public void setRolloverMaxDocs(String rolloverMaxDocs) {
+        this.rolloverMaxDocs = rolloverMaxDocs;
+    }
+
+    public void setRolloverIndexNumberOfShards(String rolloverIndexNumberOfShards) {
+        this.rolloverIndexNumberOfShards = rolloverIndexNumberOfShards;
+    }
+
+    public void setRolloverIndexNumberOfReplicas(String rolloverIndexNumberOfReplicas) {
+        this.rolloverIndexNumberOfReplicas = rolloverIndexNumberOfReplicas;
+    }
+
+    public void setRolloverIndexMappingTotalFieldsLimit(String rolloverIndexMappingTotalFieldsLimit) {
+        this.rolloverIndexMappingTotalFieldsLimit = rolloverIndexMappingTotalFieldsLimit;
+    }
+
+    public void setRolloverIndexMaxDocValueFieldsSearch(String rolloverIndexMaxDocValueFieldsSearch) {
+        this.rolloverIndexMaxDocValueFieldsSearch = rolloverIndexMaxDocValueFieldsSearch;
+    }
+
     public void setMinimalElasticSearchVersion(String minimalElasticSearchVersion) {
         this.minimalElasticSearchVersion = minimalElasticSearchVersion;
     }
@@ -410,6 +455,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
     public void setThrowExceptions(boolean throwExceptions) {
         this.throwExceptions = throwExceptions;
     }
+
     public void setAlwaysOverwrite(boolean alwaysOverwrite) {
         this.alwaysOverwrite = alwaysOverwrite;
     }
@@ -432,26 +478,21 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         new InClassLoaderExecute<Object>(null, null, this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
             public Object execute(Object... args) throws Exception {
 
-                bulkProcessorConcurrentRequests = System.getProperty(BULK_PROCESSOR_CONCURRENT_REQUESTS, bulkProcessorConcurrentRequests);
-                bulkProcessorBulkActions = System.getProperty(BULK_PROCESSOR_BULK_ACTIONS, bulkProcessorBulkActions);
-                bulkProcessorBulkSize = System.getProperty(BULK_PROCESSOR_BULK_SIZE, bulkProcessorBulkSize);
-                bulkProcessorFlushInterval = System.getProperty(BULK_PROCESSOR_FLUSH_INTERVAL, bulkProcessorFlushInterval);
-                bulkProcessorBackoffPolicy = System.getProperty(BULK_PROCESSOR_BACKOFF_POLICY, bulkProcessorBackoffPolicy);
-                itemsMonthlyIndexed = itemsMonthlyIndexedOverride.equals("none") ? Collections.emptyList() : Arrays.asList(System.getProperty(MONTHLY_INDEX_ITEMS_MONTHLY_INDEXED, itemsMonthlyIndexedOverride).split(",").clone());
-
                 buildClient();
 
                 MainResponse response = client.info(RequestOptions.DEFAULT);
-                org.elasticsearch.client.core.MainResponse.Version version = response.getVersion();
-                org.elasticsearch.Version clusterVersion = org.elasticsearch.Version.fromString(version.getNumber());
-                org.elasticsearch.Version minimalVersion = org.elasticsearch.Version.fromString(minimalElasticSearchVersion);
-                org.elasticsearch.Version maximalVersion = org.elasticsearch.Version.fromString(maximalElasticSearchVersion);
+                MainResponse.Version version = response.getVersion();
+                Version clusterVersion = Version.fromString(version.getNumber());
+                Version minimalVersion = Version.fromString(minimalElasticSearchVersion);
+                Version maximalVersion = Version.fromString(maximalElasticSearchVersion);
                 if (clusterVersion.before(minimalVersion) ||
                         clusterVersion.equals(maximalVersion) ||
                         clusterVersion.after(maximalVersion)) {
                     throw new Exception("ElasticSearch version is not within [" + minimalVersion + "," + maximalVersion + "), aborting startup !");
                 }
 
+                registerRolloverLifecyclePolicy();
+
                 loadPredefinedMappings(bundleContext, false);
 
                 // load predefined mappings and condition dispatchers of any bundles that were started before this one.
@@ -461,8 +502,6 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                     }
                 }
 
-                createMonthlyIndexTemplate();
-
                 if (client != null && bulkProcessor == null) {
                     bulkProcessor = getBulkProcessor();
                 }
@@ -677,7 +716,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         }
     }
 
-    private void loadPredefinedMappings(BundleContext bundleContext, boolean createMapping) {
+    private void loadPredefinedMappings(BundleContext bundleContext, boolean forceUpdateMapping) {
         Enumeration<URL> predefinedMappings = bundleContext.getBundle().findEntries("META-INF/cxs/mappings", "*.json", true);
         if (predefinedMappings == null) {
             return;
@@ -692,14 +731,10 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
 
                 mappings.put(name, mappingSource);
 
-                String itemIndexName = getIndex(name, new Date());
-                if (!client.indices().exists(new GetIndexRequest(itemIndexName), RequestOptions.DEFAULT)) {
-                    logger.info("{} index doesn't exist yet, creating it...", itemIndexName);
-                    internalCreateIndex(itemIndexName, mappingSource);
-                } else {
-                    logger.info("Found index {}", itemIndexName);
-                    if (createMapping) {
-                        logger.info("Updating mapping for {}", itemIndexName);
+                if (!createIndex(name)) {
+                    logger.info("Found index for type {}", name);
+                    if (forceUpdateMapping) {
+                        logger.info("Updating mapping for {}", name);
                         createMapping(name, mappingSource);
                     }
                 }
@@ -749,31 +784,36 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
 
     @Override
     public <T extends Item> T load(final String itemId, final Class<T> clazz) {
-        return load(itemId, null, clazz);
+        return load(itemId, clazz, null);
     }
 
     @Override
+    @Deprecated
     public <T extends Item> T load(final String itemId, final Date dateHint, final Class<T> clazz) {
-        return load(itemId, dateHint, clazz, null);
+        return load(itemId, clazz, null);
     }
 
     @Override
+    @Deprecated
     public CustomItem loadCustomItem(final String itemId, final Date dateHint, String customItemType) {
-        return load(itemId, dateHint, CustomItem.class, customItemType);
+        return load(itemId, CustomItem.class, customItemType);
     }
 
-    private <T extends Item> T load(final String itemId, final Date dateHint, final Class<T> clazz, final String customItemType) {
+    @Override
+    public CustomItem loadCustomItem(final String itemId, String customItemType) {
+        return load(itemId, CustomItem.class, customItemType);
+    }
+
+    private <T extends Item> T load(final String itemId, final Class<T> clazz, final String customItemType) {
         return new InClassLoaderExecute<T>(metricsService, this.getClass().getName() + ".loadItem", this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
             protected T execute(Object... args) throws Exception {
                 try {
                     String itemType = Item.getItemType(clazz);
-                    String className = clazz.getName();
                     if (customItemType != null) {
-                        className = CustomItem.class.getName() + "." + customItemType;
                         itemType = customItemType;
                     }
 
-                    if (itemsMonthlyIndexed.contains(itemType) && dateHint == null) {
+                    if (isItemTypeRollingOver(itemType)) {
                         return new MetricAdapter<T>(metricsService, ".loadItemWithQuery") {
                             @Override
                             public T execute(Object... args) throws Exception {
@@ -792,12 +832,12 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                             }
                         }.execute();
                     } else {
-                        GetRequest getRequest = new GetRequest(getIndex(itemType, dateHint), itemId);
+                        GetRequest getRequest = new GetRequest(getIndex(itemType), itemId);
                         GetResponse response = client.get(getRequest, RequestOptions.DEFAULT);
                         if (response.isExists()) {
                             String sourceAsString = response.getSourceAsString();
                             final T value = ESCustomObjectMapper.getObjectMapper().readValue(sourceAsString, clazz);
-                            setMetadata(value, response.getId(), response.getVersion(), response.getSeqNo(), response.getPrimaryTerm());
+                            setMetadata(value, response.getId(), response.getVersion(), response.getSeqNo(), response.getPrimaryTerm(), response.getIndex());
                             return value;
                         } else {
                             return null;
@@ -813,18 +853,19 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                     // this can happen if we are just testing the existence of the item, it is not always an error.
                     return null;
                 } catch (Exception ex) {
-                    throw new Exception("Error loading itemType=" + clazz.getName() + " customItemType=" + customItemType+ " itemId=" + itemId, ex);
+                    throw new Exception("Error loading itemType=" + clazz.getName() + " customItemType=" + customItemType + " itemId=" + itemId, ex);
                 }
             }
         }.catchingExecuteInClassLoader(true);
 
     }
 
-    private void setMetadata(Item item, String id, long version, long seqNo, long primaryTerm) {
+    private void setMetadata(Item item, String id, long version, long seqNo, long primaryTerm, String index) {
         item.setItemId(id);
         item.setVersion(version);
         item.setSystemMetadata(SEQ_NO, seqNo);
         item.setSystemMetadata(PRIMARY_TERM, primaryTerm);
+        item.setSystemMetadata("index", index);
     }
 
     @Override
@@ -858,20 +899,21 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                         className = CustomItem.class.getName() + "." + itemType;
                     }
                     String itemId = item.getItemId();
-                    String index = getIndex(itemType, itemsMonthlyIndexed.contains(itemType) ? ((TimestampedItem) item).getTimeStamp() : null);
+                    String index = item.getSystemMetadata("index") != null ?
+                            (String) item.getSystemMetadata("index") :
+                            getIndex(itemType);
                     IndexRequest indexRequest = new IndexRequest(index);
                     indexRequest.id(itemId);
                     indexRequest.source(source, XContentType.JSON);
 
                     if (!alwaysOverwrite) {
-                        Long seqNo = (Long)item.getSystemMetadata(SEQ_NO);
-                        Long primaryTerm = (Long)item.getSystemMetadata(PRIMARY_TERM);
+                        Long seqNo = (Long) item.getSystemMetadata(SEQ_NO);
+                        Long primaryTerm = (Long) item.getSystemMetadata(PRIMARY_TERM);
 
                         if (seqNo != null && primaryTerm != null) {
                             indexRequest.setIfSeqNo(seqNo);
                             indexRequest.setIfPrimaryTerm(primaryTerm);
-                        }
-                        else {
+                        } else {
                             indexRequest.opType(DocWriteRequest.OpType.CREATE);
                         }
                     }
@@ -884,7 +926,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                         if (bulkProcessor == null || !useBatching) {
                             indexRequest.setRefreshPolicy(getRefreshPolicy(itemType));
                             IndexResponse response = client.index(indexRequest, RequestOptions.DEFAULT);
-                            setMetadata(item, response.getId(), response.getVersion(), response.getSeqNo(), response.getPrimaryTerm());
+                            setMetadata(item, response.getId(), response.getVersion(), response.getSeqNo(), response.getPrimaryTerm(), response.getIndex());
                         } else {
                             bulkProcessor.add(indexRequest);
                         }
@@ -908,24 +950,40 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
 
     @Override
     public boolean update(final Item item, final Date dateHint, final Class clazz, final String propertyName, final Object propertyValue) {
-        return update(item, dateHint, clazz, Collections.singletonMap(propertyName, propertyValue));
+        return update(item, clazz, propertyName, propertyValue);
     }
 
     @Override
     public boolean update(final Item item, final Date dateHint, final Class clazz, final Map source) {
-        return update(item, dateHint, clazz, source, alwaysOverwrite);
+        return update(item, clazz, source);
     }
 
     @Override
     public boolean update(final Item item, final Date dateHint, final Class clazz, final Map source, final boolean alwaysOverwrite) {
+        return update(item, clazz, source, alwaysOverwrite);
+    }
+
+    @Override
+    public boolean update(final Item item, final Class clazz, final String propertyName, final Object propertyValue) {
+        return update(item, clazz, Collections.singletonMap(propertyName, propertyValue), alwaysOverwrite);
+    }
+
+
+    @Override
+    public boolean update(final Item item, final Class clazz, final Map source) {
+        return update(item, clazz, source, alwaysOverwrite);
+    }
+
+    @Override
+    public boolean update(final Item item, final Class clazz, final Map source, final boolean alwaysOverwrite) {
         Boolean result = new InClassLoaderExecute<Boolean>(metricsService, this.getClass().getName() + ".updateItem", this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
             protected Boolean execute(Object... args) throws Exception {
                 try {
-                    UpdateRequest updateRequest = createUpdateRequest(clazz, dateHint, item, source, alwaysOverwrite);
+                    UpdateRequest updateRequest = createUpdateRequest(clazz, item, source, alwaysOverwrite);
 
                     if (bulkProcessor == null || !useBatchingForUpdate) {
                         UpdateResponse response = client.update(updateRequest, RequestOptions.DEFAULT);
-                        setMetadata(item, response.getId(), response.getVersion(), response.getSeqNo(), response.getPrimaryTerm());
+                        setMetadata(item, response.getId(), response.getVersion(), response.getSeqNo(), response.getPrimaryTerm(), response.getIndex());
                     } else {
                         bulkProcessor.add(updateRequest);
                     }
@@ -942,9 +1000,9 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         }
     }
 
-    private UpdateRequest createUpdateRequest(Class clazz, Date dateHint, Item item, Map source, boolean alwaysOverwrite) {
+    private UpdateRequest createUpdateRequest(Class clazz, Item item, Map source, boolean alwaysOverwrite) {
         String itemType = Item.getItemType(clazz);
-        UpdateRequest updateRequest = new UpdateRequest(getIndex(itemType, dateHint), item.getItemId());
+        UpdateRequest updateRequest = new UpdateRequest(getIndex(itemType), item.getItemId());
         updateRequest.doc(source);
 
         if (!alwaysOverwrite) {
@@ -964,13 +1022,13 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         if (items.size() == 0)
             return new ArrayList<>();
 
-        List<String> result = new InClassLoaderExecute<List<String>>(metricsService, this.getClass().getName() + ".updateItems",  this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
+        List<String> result = new InClassLoaderExecute<List<String>>(metricsService, this.getClass().getName() + ".updateItems", this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
             protected List<String> execute(Object... args) throws Exception {
                 long batchRequestStartTime = System.currentTimeMillis();
 
                 BulkRequest bulkRequest = new BulkRequest();
                 items.forEach((item, source) -> {
-                    UpdateRequest updateRequest = createUpdateRequest(clazz, dateHint, item, source, alwaysOverwrite);
+                    UpdateRequest updateRequest = createUpdateRequest(clazz, item, source, alwaysOverwrite);
                     bulkRequest.add(updateRequest);
                 });
 
@@ -979,7 +1037,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
 
                 List<String> failedItemsIds = new ArrayList<>();
 
-                if (bulkResponse.hasFailures()){
+                if (bulkResponse.hasFailures()) {
                     Iterator<BulkItemResponse> iterator = bulkResponse.iterator();
                     iterator.forEachRemaining(bulkItemResponse -> {
                         failedItemsIds.add(bulkItemResponse.getId());
@@ -992,32 +1050,41 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         return result;
     }
 
-
     @Override
     public boolean updateWithQueryAndScript(final Date dateHint, final Class<?> clazz, final String[] scripts, final Map<String, Object>[] scriptParams, final Condition[] conditions) {
+        return updateWithQueryAndScript(clazz, scripts, scriptParams, conditions);
+    }
+
+    @Override
+    public boolean updateWithQueryAndScript(final Class<?> clazz, final String[] scripts, final Map<String, Object>[] scriptParams, final Condition[] conditions) {
         Script[] builtScripts = new Script[scripts.length];
         for (int i = 0; i < scripts.length; i++) {
             builtScripts[i] = new Script(ScriptType.INLINE, "painless", scripts[i], scriptParams[i]);
         }
-        return updateWithQueryAndScript(dateHint, clazz, builtScripts, conditions);
+        return updateWithQueryAndScript(clazz, builtScripts, conditions);
     }
 
     @Override
     public boolean updateWithQueryAndStoredScript(Date dateHint, Class<?> clazz, String[] scripts, Map<String, Object>[] scriptParams, Condition[] conditions) {
+        return updateWithQueryAndStoredScript(clazz, scripts, scriptParams, conditions);
+    }
+
+    @Override
+    public boolean updateWithQueryAndStoredScript(Class<?> clazz, String[] scripts, Map<String, Object>[] scriptParams, Condition[] conditions) {
         Script[] builtScripts = new Script[scripts.length];
         for (int i = 0; i < scripts.length; i++) {
             builtScripts[i] = new Script(ScriptType.STORED, null, scripts[i], scriptParams[i]);
         }
-        return updateWithQueryAndScript(dateHint, clazz, builtScripts, conditions);
+        return updateWithQueryAndScript(clazz, builtScripts, conditions);
     }
 
-    private boolean updateWithQueryAndScript(final Date dateHint, final Class<?> clazz, final Script[] scripts, final Condition[] conditions) {
+    private boolean updateWithQueryAndScript(final Class<?> clazz, final Script[] scripts, final Condition[] conditions) {
         Boolean result = new InClassLoaderExecute<Boolean>(metricsService, this.getClass().getName() + ".updateWithQueryAndScript", this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
             protected Boolean execute(Object... args) throws Exception {
                 try {
                     String itemType = Item.getItemType(clazz);
 
-                    String index = getIndex(itemType, dateHint);
+                    String index = getIndex(itemType);
 
                     for (int i = 0; i < scripts.length; i++) {
                         RefreshRequest refreshRequest = new RefreshRequest(index);
@@ -1103,21 +1170,25 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         }
     }
 
-    @Override
     public boolean updateWithScript(final Item item, final Date dateHint, final Class<?> clazz, final String script, final Map<String, Object> scriptParams) {
+        return updateWithScript(item, clazz, script, scriptParams);
+    }
+
+    @Override
+    public boolean updateWithScript(final Item item, final Class<?> clazz, final String script, final Map<String, Object> scriptParams) {
         Boolean result = new InClassLoaderExecute<Boolean>(metricsService, this.getClass().getName() + ".updateWithScript", this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
             protected Boolean execute(Object... args) throws Exception {
                 try {
                     String itemType = Item.getItemType(clazz);
 
-                    String index = getIndex(itemType, dateHint);
+                    String index = getIndex(itemType);
 
                     Script actualScript = new Script(ScriptType.INLINE, "painless", script, scriptParams);
 
                     UpdateRequest updateRequest = new UpdateRequest(index, item.getItemId());
 
-                    Long seqNo = (Long)item.getSystemMetadata(SEQ_NO);
-                    Long primaryTerm = (Long)item.getSystemMetadata(PRIMARY_TERM);
+                    Long seqNo = (Long) item.getSystemMetadata(SEQ_NO);
+                    Long primaryTerm = (Long) item.getSystemMetadata(PRIMARY_TERM);
 
                     if (seqNo != null && primaryTerm != null) {
                         updateRequest.setIfSeqNo(seqNo);
@@ -1126,7 +1197,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                     updateRequest.script(actualScript);
                     if (bulkProcessor == null) {
                         UpdateResponse response = client.update(updateRequest, RequestOptions.DEFAULT);
-                        setMetadata(item, response.getId(), response.getVersion(), response.getSeqNo(), response.getPrimaryTerm());
+                        setMetadata(item, response.getId(), response.getVersion(), response.getSeqNo(), response.getPrimaryTerm(), response.getIndex());
                     } else {
                         bulkProcessor.add(updateRequest);
                     }
@@ -1281,40 +1352,31 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         }
     }
 
-    public boolean createMonthlyIndexTemplate() {
-        Boolean result = new InClassLoaderExecute<Boolean>(metricsService, this.getClass().getName() + ".createMonthlyIndexTemplate", this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
+    public boolean registerRolloverLifecyclePolicy() {
+        Boolean result = new InClassLoaderExecute<Boolean>(metricsService, this.getClass().getName() + ".createMonthlyIndexLifecyclePolicy", this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
             protected Boolean execute(Object... args) throws IOException {
-                boolean executedSuccessfully = true;
-                for (String itemName : itemsMonthlyIndexed) {
-                    PutIndexTemplateRequest putIndexTemplateRequest = new PutIndexTemplateRequest(indexPrefix + "-" + itemName + "-date-template")
-                            .patterns(Collections.singletonList(getMonthlyIndexForQuery(itemName)))
-                            .order(1)
-                            .settings("{\n" +
-                                    "    \"index\" : {\n" +
-                                    "        \"number_of_shards\" : " + monthlyIndexNumberOfShards + ",\n" +
-                                    "        \"number_of_replicas\" : " + monthlyIndexNumberOfReplicas + ",\n" +
-                                    "        \"mapping.total_fields.limit\" : " + monthlyIndexMappingTotalFieldsLimit + ",\n" +
-                                    "        \"max_docvalue_fields_search\" : " + monthlyIndexMaxDocValueFieldsSearch + "\n" +
-                                    "    },\n" +
-                                    "    \"analysis\": {\n" +
-                                    "      \"analyzer\": {\n" +
-                                    "        \"folding\": {\n" +
-                                    "          \"type\":\"custom\",\n" +
-                                    "          \"tokenizer\": \"keyword\",\n" +
-                                    "          \"filter\":  [ \"lowercase\", \"asciifolding\" ]\n" +
-                                    "        }\n" +
-                                    "      }\n" +
-                                    "    }\n" +
-                                    "}\n", XContentType.JSON);
-                    if (mappings.get(itemName) == null) {
-                        logger.warn("Couldn't find mapping for item {}, won't create monthly index template", itemName);
-                        return false;
-                    }
-                    putIndexTemplateRequest.mapping(mappings.get(itemName), XContentType.JSON);
-                    AcknowledgedResponse putIndexTemplateResponse = client.indices().putTemplate(putIndexTemplateRequest, RequestOptions.DEFAULT);
-                    executedSuccessfully &= putIndexTemplateResponse.isAcknowledged();
-                }
-                return executedSuccessfully;
+                // Create the lifecycle policy for monthly indices
+                Map<String, Phase> phases = new HashMap<>();
+                Map<String, LifecycleAction> hotActions = new HashMap<>();
+                final Long maxDocs = StringUtils.isEmpty(rolloverMaxDocs) ? null : Long.parseLong(rolloverMaxDocs);
+                hotActions.put(
+                        RolloverAction.NAME,
+                        new RolloverAction(
+                                StringUtils.isEmpty(rolloverMaxSize) ? null : ByteSizeValue.parseBytesSizeValue(rolloverMaxSize, "rollover.maxSize"),
+                                StringUtils.isEmpty(rolloverMaxAge) ? null : TimeValue.parseTimeValue(rolloverMaxAge, null, "rollover.maxAge"),
+                                maxDocs
+                        )
+                );
+                phases.put("hot", new Phase("hot", TimeValue.ZERO, hotActions));
+
+                // TODO - Handle this with the purge https://issues.apache.org/jira/browse/UNOMI-726
+                Map<String, LifecycleAction> deleteActions = Collections.singletonMap(DeleteAction.NAME, new DeleteAction());
+                phases.put("delete", new Phase("delete", new TimeValue(90, TimeUnit.DAYS), deleteActions));
+
+                LifecyclePolicy policy = new LifecyclePolicy(indexPrefix + "-" + ROLLOVER_LIFECYCLE_NAME, phases);
+                PutLifecyclePolicyRequest request = new PutLifecyclePolicyRequest(policy);
+                org.elasticsearch.client.core.AcknowledgedResponse putLifecyclePolicy = client.indexLifecycle().putLifecyclePolicy(request, RequestOptions.DEFAULT);
+                return putLifecyclePolicy.isAcknowledged();
             }
         }.catchingExecuteInClassLoader(true);
         if (result == null) {
@@ -1325,15 +1387,22 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
     }
 
     public boolean createIndex(final String itemType) {
-        String index = getIndex(itemType);
 
         Boolean result = new InClassLoaderExecute<Boolean>(metricsService, this.getClass().getName() + ".createIndex", this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
             protected Boolean execute(Object... args) throws IOException {
+                String index = getIndex(itemType);
                 GetIndexRequest getIndexRequest = new GetIndexRequest(index);
                 boolean indexExists = client.indices().exists(getIndexRequest, RequestOptions.DEFAULT);
+
                 if (!indexExists) {
-                    internalCreateIndex(index, mappings.get(itemType));
+                    if (isItemTypeRollingOver(itemType)) {
+                        internalCreateRolloverTemplate(itemType);
+                        internalCreateRolloverIndex(index);
+                    } else {
+                        internalCreateIndex(index, mappings.get(itemType));
+                    }
                 }
+
                 return !indexExists;
             }
         }.catchingExecuteInClassLoader(true);
@@ -1367,6 +1436,47 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
         }
     }
 
+    private void internalCreateRolloverTemplate(String itemName) throws IOException {
+        String rolloverAlias = indexPrefix + "-" + itemName;
+        PutIndexTemplateRequest putIndexTemplateRequest = new PutIndexTemplateRequest(rolloverAlias + "-rollover-template")
+                .patterns(Collections.singletonList(getRolloverIndexForQuery(itemName)))
+                .order(1)
+                .settings("{\n" +
+                        "    \"index\" : {\n" +
+                        "        \"number_of_shards\" : " + StringUtils.defaultIfEmpty(rolloverIndexNumberOfShards, monthlyIndexNumberOfShards) + ",\n" +
+                        "        \"number_of_replicas\" : " + StringUtils.defaultIfEmpty(rolloverIndexNumberOfReplicas, monthlyIndexNumberOfReplicas) + ",\n" +
+                        "        \"mapping.total_fields.limit\" : " + StringUtils.defaultIfEmpty(rolloverIndexMappingTotalFieldsLimit, monthlyIndexMappingTotalFieldsLimit) + ",\n" +
+                        "        \"max_docvalue_fields_search\" : " + StringUtils.defaultIfEmpty(rolloverIndexMaxDocValueFieldsSearch, monthlyIndexMaxDocValueFieldsSearch) + ",\n" +
+                        "        \"lifecycle.name\": \"" + (indexPrefix + "-" + ROLLOVER_LIFECYCLE_NAME) + "\",\n" +
+                        "        \"lifecycle.rollover_alias\": \"" + rolloverAlias + "\"" +
+                        "" +
+                        "    },\n" +
+                        "    \"analysis\": {\n" +
+                        "      \"analyzer\": {\n" +
+                        "        \"folding\": {\n" +
+                        "          \"type\":\"custom\",\n" +
+                        "          \"tokenizer\": \"keyword\",\n" +
+                        "          \"filter\":  [ \"lowercase\", \"asciifolding\" ]\n" +
+                        "        }\n" +
+                        "      }\n" +
+                        "    }\n" +
+                        "}\n", XContentType.JSON);
+        if (mappings.get(itemName) == null) {
+            logger.warn("Couldn't find mapping for item {}, won't create monthly index template", itemName);
+            return;
+        }
+        putIndexTemplateRequest.mapping(mappings.get(itemName), XContentType.JSON);
+        client.indices().putTemplate(putIndexTemplateRequest, RequestOptions.DEFAULT);
+    }
+
+    private void internalCreateRolloverIndex(String indexName) throws IOException {
+        CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName + "-000001")
+                .alias(new Alias(indexName).writeIndex(true));
+        CreateIndexResponse createIndexResponse = client.indices().create(createIndexRequest, RequestOptions.DEFAULT);
+        logger.info("Index created: [{}], acknowledge: [{}], shards acknowledge: [{}]", createIndexResponse.index(),
+                createIndexResponse.isAcknowledged(), createIndexResponse.isShardsAcknowledged());
+    }
+
     private void internalCreateIndex(String indexName, String mappingSource) throws IOException {
         CreateIndexRequest createIndexRequest = new CreateIndexRequest(indexName);
         createIndexRequest.settings("{\n" +
@@ -1398,16 +1508,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
     @Override
     public void createMapping(String type, String source) {
         try {
-            if (itemsMonthlyIndexed.contains(type)) {
-                createMonthlyIndexTemplate();
-                String indexName = getIndex(type, new Date());
-                GetIndexRequest getIndexRequest = new GetIndexRequest(indexName);
-                if (client.indices().exists(getIndexRequest, RequestOptions.DEFAULT)) {
-                    putMapping(source, indexName);
-                }
-            } else {
-                putMapping(source, getIndex(type));
-            }
+            putMapping(source, getIndex(type));
         } catch (IOException ioe) {
             logger.error("Error while creating mapping for type " + type + " and source " + source, ioe);
         }
@@ -1611,7 +1712,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                 //Index the query = register it in the percolator
                 try {
                     logger.info("Saving query : " + queryName);
-                    String index = getIndex(".percolator", null);
+                    String index = getIndex(".percolator");
                     IndexRequest indexRequest = new IndexRequest(index);
                     indexRequest.id(queryName);
                     indexRequest.source(query, XContentType.JSON);
@@ -1645,7 +1746,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
             protected Boolean execute(Object... args) throws Exception {
                 //Index the query = register it in the percolator
                 try {
-                    String index = getIndex(".percolator", null);
+                    String index = getIndex(".percolator");
                     DeleteRequest deleteRequest = new DeleteRequest(index);
                     deleteRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
                     client.delete(deleteRequest, RequestOptions.DEFAULT);
@@ -1884,7 +1985,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                                 // add hit to results
                                 String sourceAsString = searchHit.getSourceAsString();
                                 final T value = ESCustomObjectMapper.getObjectMapper().readValue(sourceAsString, clazz);
-                                setMetadata(value, searchHit.getId(), searchHit.getVersion(), searchHit.getSeqNo(), searchHit.getPrimaryTerm());
+                                setMetadata(value, searchHit.getId(), searchHit.getVersion(), searchHit.getSeqNo(), searchHit.getPrimaryTerm(), searchHit.getIndex());
                                 results.add(value);
                             }
 
@@ -1914,7 +2015,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                         for (SearchHit searchHit : searchHits) {
                             String sourceAsString = searchHit.getSourceAsString();
                             final T value = ESCustomObjectMapper.getObjectMapper().readValue(sourceAsString, clazz);
-                            setMetadata(value, searchHit.getId(), searchHit.getVersion(), searchHit.getSeqNo(), searchHit.getPrimaryTerm());
+                            setMetadata(value, searchHit.getId(), searchHit.getVersion(), searchHit.getSeqNo(), searchHit.getPrimaryTerm(), searchHit.getIndex());
                             results.add(value);
                         }
                     }
@@ -1960,7 +2061,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                             // add hit to results
                             String sourceAsString = searchHit.getSourceAsString();
                             final T value = ESCustomObjectMapper.getObjectMapper().readValue(sourceAsString, clazz);
-                            setMetadata(value, searchHit.getId(), searchHit.getVersion(), searchHit.getSeqNo(), searchHit.getPrimaryTerm());
+                            setMetadata(value, searchHit.getId(), searchHit.getVersion(), searchHit.getSeqNo(), searchHit.getPrimaryTerm(), searchHit.getIndex());
                             results.add(value);
                         }
                     }
@@ -2001,7 +2102,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                             // add hit to results
                             String sourceAsString = searchHit.getSourceAsString();
                             final CustomItem value = ESCustomObjectMapper.getObjectMapper().readValue(sourceAsString, CustomItem.class);
-                            setMetadata(value, searchHit.getId(), searchHit.getVersion(), searchHit.getSeqNo(), searchHit.getPrimaryTerm());
+                            setMetadata(value, searchHit.getId(), searchHit.getVersion(), searchHit.getSeqNo(), searchHit.getPrimaryTerm(), searchHit.getIndex());
                             results.add(value);
                         }
                     }
@@ -2038,7 +2139,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
     }
 
     private Map<String, Long> aggregateQuery(final Condition filter, final BaseAggregate aggregate, final String itemType,
-            final boolean optimizedQuery, int queryBucketSize) {
+                                             final boolean optimizedQuery, int queryBucketSize) {
         return new InClassLoaderExecute<Map<String, Long>>(metricsService, this.getClass().getName() + ".aggregateQuery", this.bundleContext, this.fatalIllegalStateErrors, throwExceptions) {
 
             @Override
@@ -2244,7 +2345,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
             protected Boolean execute(Object... args) {
                 try {
                     String itemType = Item.getItemType(clazz);
-                    String index = getIndex(itemType, dateHint);
+                    String index = getIndex(itemType);
                     client.indices().refresh(Requests.refreshRequest(index), RequestOptions.DEFAULT);
                 } catch (IOException e) {
                     e.printStackTrace();//TODO manage ES7
@@ -2485,25 +2586,19 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
     }
 
     private String getIndexNameForQuery(String itemType) {
-        return itemsMonthlyIndexed.contains(itemType) ? getMonthlyIndexForQuery(itemType) : getIndex(itemType, null);
-    }
-
-    private String getMonthlyIndexForQuery(String itemType) {
-        return indexPrefix + "-" + itemType.toLowerCase() + "-" + INDEX_DATE_PREFIX + "*";
+        return isItemTypeRollingOver(itemType) ? getRolloverIndexForQuery(itemType) : getIndex(itemType);
     }
 
-    private String getIndex(String itemType, Date dateHint) {
-        String indexItemTypePart = itemsMonthlyIndexed.contains(itemType) && dateHint != null ? itemType + "-" + getMonthlyIndexPart(dateHint) : itemType;
-        return getIndex(indexItemTypePart);
+    private String getRolloverIndexForQuery(String itemType) {
+        return indexPrefix + "-" + itemType.toLowerCase() + "-*";
     }
 
     private String getIndex(String indexItemTypePart) {
         return (indexPrefix + "-" + indexItemTypePart).toLowerCase();
     }
 
-    private String getMonthlyIndexPart(Date date) {
-        String d = new SimpleDateFormat("yyyy-MM").format(date);
-        return INDEX_DATE_PREFIX + d;
+    private boolean isItemTypeRollingOver(String itemType) {
+        return (rolloverIndices != null ? rolloverIndices : itemsMonthlyIndexed).contains(itemType);
     }
 
     private WriteRequest.RefreshPolicy getRefreshPolicy(String itemType) {
diff --git a/persistence-elasticsearch/core/src/main/resources/OSGI-INF/blueprint/blueprint.xml b/persistence-elasticsearch/core/src/main/resources/OSGI-INF/blueprint/blueprint.xml
index e0fdded38..83d4ecd41 100644
--- a/persistence-elasticsearch/core/src/main/resources/OSGI-INF/blueprint/blueprint.xml
+++ b/persistence-elasticsearch/core/src/main/resources/OSGI-INF/blueprint/blueprint.xml
@@ -50,6 +50,15 @@
             <cm:property name="bulkProcessor.flushInterval" value="5s" />
             <cm:property name="bulkProcessor.backoffPolicy" value="exponential" />
 
+            <cm:property name="rollover.indices" value="" />
+            <cm:property name="rollover.maxSize" value="" />
+            <cm:property name="rollover.maxAge" value="365d" />
+            <cm:property name="rollover.maxDocs" value="" />
+            <cm:property name="rollover.numberOfShards" value=""/>
+            <cm:property name="rollover.numberOfReplicas" value=""/>
+            <cm:property name="rollover.indexMappingTotalFieldsLimit" value=""/>
+            <cm:property name="rollover.indexMaxDocValueFieldsSearch" value=""/>
+
             <cm:property name="minimalElasticSearchVersion" value="7.0.0" />
             <cm:property name="maximalElasticSearchVersion" value="8.0.0" />
 
@@ -124,6 +133,15 @@
         <property name="bulkProcessorFlushInterval" value="${es.bulkProcessor.flushInterval}" />
         <property name="bulkProcessorBackoffPolicy" value="${es.bulkProcessor.backoffPolicy}" />
 
+        <property name="rolloverIndices" value="${es.rollover.indices}" />
+        <property name="rolloverMaxSize" value="${es.rollover.maxSize}" />
+        <property name="rolloverMaxAge" value="${es.rollover.maxAge}" />
+        <property name="rolloverMaxDocs" value="${es.rollover.maxDocs}" />
+        <property name="rolloverIndexNumberOfShards" value="${es.rollover.numberOfShards}"/>
+        <property name="rolloverIndexNumberOfReplicas" value="${es.rollover.numberOfReplicas}"/>
+        <property name="rolloverIndexMappingTotalFieldsLimit" value="${es.rollover.indexMappingTotalFieldsLimit}"/>
+        <property name="rolloverIndexMaxDocValueFieldsSearch" value="${es.rollover.indexMaxDocValueFieldsSearch}"/>
+
         <property name="minimalElasticSearchVersion" value="${es.minimalElasticSearchVersion}" />
         <property name="maximalElasticSearchVersion" value="${es.maximalElasticSearchVersion}" />
 
diff --git a/persistence-elasticsearch/core/src/main/resources/org.apache.unomi.persistence.elasticsearch.cfg b/persistence-elasticsearch/core/src/main/resources/org.apache.unomi.persistence.elasticsearch.cfg
index b996c2a1c..9d055cff6 100644
--- a/persistence-elasticsearch/core/src/main/resources/org.apache.unomi.persistence.elasticsearch.cfg
+++ b/persistence-elasticsearch/core/src/main/resources/org.apache.unomi.persistence.elasticsearch.cfg
@@ -22,16 +22,32 @@ cluster.name=${org.apache.unomi.elasticsearch.cluster.name:-contextElasticSearch
 elasticSearchAddresses=${org.apache.unomi.elasticsearch.addresses:-localhost:9200}
 fatalIllegalStateErrors=${org.apache.unomi.elasticsearch.fatalIllegalStateErrors:-}
 index.prefix=${org.apache.unomi.elasticsearch.index.prefix:-context}
+
+# Deprecated properties. Please use rollover corresponding properties
 monthlyIndex.numberOfShards=${org.apache.unomi.elasticsearch.monthlyIndex.nbShards:-5}
 monthlyIndex.numberOfReplicas=${org.apache.unomi.elasticsearch.monthlyIndex.nbReplicas:-0}
 monthlyIndex.indexMappingTotalFieldsLimit=${org.apache.unomi.elasticsearch.monthlyIndex.indexMappingTotalFieldsLimit:-1000}
 monthlyIndex.indexMaxDocValueFieldsSearch=${org.apache.unomi.elasticsearch.monthlyIndex.indexMaxDocValueFieldsSearch:-1000}
+monthlyIndex.itemsMonthlyIndexedOverride=${org.apache.unomi.elasticsearch.monthlyIndex.itemsMonthlyIndexedOverride:-event,session}
+# New properties for index rotation:
+rollover.numberOfShards=${org.apache.unomi.elasticsearch.rollover.nbShards}
+rollover.numberOfReplicas=${org.apache.unomi.elasticsearch.rollover.nbReplicas}
+rollover.indexMappingTotalFieldsLimit=${org.apache.unomi.elasticsearch.rollover.indexMappingTotalFieldsLimit}
+rollover.indexMaxDocValueFieldsSearch=${org.apache.unomi.elasticsearch.rollover.indexMaxDocValueFieldsSearch}
+rollover.indices=${org.apache.unomi.elasticsearch.rollover.indices}
+
 numberOfShards=${org.apache.unomi.elasticsearch.defaultIndex.nbShards:-5}
 numberOfReplicas=${org.apache.unomi.elasticsearch.defaultIndex.nbReplicas:-0}
 indexMappingTotalFieldsLimit=${org.apache.unomi.elasticsearch.defaultIndex.indexMappingTotalFieldsLimit:-1000}
 indexMaxDocValueFieldsSearch=${org.apache.unomi.elasticsearch.defaultIndex.indexMaxDocValueFieldsSearch:-1000}
 defaultQueryLimit=${org.apache.unomi.elasticsearch.defaultQueryLimit:-10}
 
+# Rollover amd index configuration for event and session indices, values are cumulative
+# See https://www.elastic.co/guide/en/elasticsearch/reference/7.17/ilm-rollover.html for option details.
+rollover.maxSize=${org.apache.unomi.elasticsearch.rollover.maxSize}
+rollover.maxAge=${org.apache.unomi.elasticsearch.rollover.maxAge:-365d}
+rollover.maxDocs=${org.apache.unomi.elasticsearch.rollover.maxDocs}
+
 # The following settings control the behavior of the BulkProcessor API. You can find more information about these
 # settings and their behavior here : https://www.elastic.co/guide/en/elasticsearch/client/java-api/2.4/java-docs-bulk-processor.html
 # The values used here are the default values of the API
diff --git a/persistence-elasticsearch/core/src/test/java/org/apache/unomi/persistence/elasticsearch/ElasticsearchPersistenceTest.java b/persistence-elasticsearch/core/src/test/java/org/apache/unomi/persistence/elasticsearch/ElasticsearchPersistenceTest.java
index 192bcb1e6..c80734f94 100644
--- a/persistence-elasticsearch/core/src/test/java/org/apache/unomi/persistence/elasticsearch/ElasticsearchPersistenceTest.java
+++ b/persistence-elasticsearch/core/src/test/java/org/apache/unomi/persistence/elasticsearch/ElasticsearchPersistenceTest.java
@@ -45,6 +45,7 @@ import org.junit.runner.RunWith;
 
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Date;
 import java.util.UUID;
 import java.util.logging.Logger;
 
@@ -126,7 +127,8 @@ public class ElasticsearchPersistenceTest {
     @Test
     public void testCreateIndex() throws Exception {
         restHighLevelClient.info(RequestOptions.DEFAULT.toBuilder().addHeader("name", "value").build());
-        CreateIndexRequest request = new CreateIndexRequest("unomi-index-1");
+        final String indexName = "unomi-index-" + new Date().getTime();
+        CreateIndexRequest request = new CreateIndexRequest(indexName);
         CreateIndexResponse response = restHighLevelClient.indices().create(request, RequestOptions.DEFAULT);
         if (response.isAcknowledged()) {
             logger.info(">>> Create index :: ok :: name = " + response.index());
@@ -152,7 +154,7 @@ public class ElasticsearchPersistenceTest {
 //        }
 //        Assert.assertNotEquals(actionGet.getStatus(), ClusterHealthStatus.RED);
 
-        IndexRequest indexRequest = new IndexRequest("unomi-index-1");
+        IndexRequest indexRequest = new IndexRequest(indexName);
         indexRequest.id(UUID.randomUUID().toString());
         String type = "{\"type\":\"unomi-type\"}";
         String source = "{\"name\":\"unomi-name\"}";
diff --git a/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/PersistenceService.java b/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/PersistenceService.java
index 625a5948e..d0911131d 100644
--- a/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/PersistenceService.java
+++ b/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/PersistenceService.java
@@ -47,7 +47,7 @@ public interface PersistenceService {
     /**
      * Retrieves all known items of the specified class, ordered according to the specified {@code sortBy} String and and paged: only {@code size} of them are retrieved,
      * starting with the {@code offset}-th one.
-     *
+     * <p>
      * TODO: use a Query object instead of distinct parameters?
      *
      * @param <T>    the type of the {@link Item}s we want to retrieve
@@ -65,19 +65,19 @@ public interface PersistenceService {
     /**
      * Retrieves all known items of the specified class, ordered according to the specified {@code sortBy} String and and paged: only {@code size} of them are retrieved,
      * starting with the {@code offset}-th one.
-     *
+     * <p>
      * TODO: use a Query object instead of distinct parameters?
      *
-     * @param <T>    the type of the {@link Item}s we want to retrieve
-     * @param clazz  the {@link Item} subclass of entities we want to retrieve
-     * @param offset zero or a positive integer specifying the position of the first item in the total ordered collection of matching items
-     * @param size   a positive integer specifying how many matching items should be retrieved or {@code -1} if all of them should be retrieved
-     * @param sortBy an optional ({@code null} if no sorting is required) String of comma ({@code ,}) separated property names on which ordering should be performed, ordering
-     *               elements according to the property order in the
-     *               String, considering each in turn and moving on to the next one in case of equality of all preceding ones. Each property name is optionally followed by
-     *               a column ({@code :}) and an order specifier: {@code asc} or {@code desc}.
-     * @param scrollTimeValidity  the time the scrolling query should stay valid. This must contain a time unit value such as the ones supported by ElasticSearch, such as
-     *      *                     the ones declared here : https://www.elastic.co/guide/en/elasticsearch/reference/current/common-options.html#time-units
+     * @param <T>                the type of the {@link Item}s we want to retrieve
+     * @param clazz              the {@link Item} subclass of entities we want to retrieve
+     * @param offset             zero or a positive integer specifying the position of the first item in the total ordered collection of matching items
+     * @param size               a positive integer specifying how many matching items should be retrieved or {@code -1} if all of them should be retrieved
+     * @param sortBy             an optional ({@code null} if no sorting is required) String of comma ({@code ,}) separated property names on which ordering should be performed, ordering
+     *                           elements according to the property order in the
+     *                           String, considering each in turn and moving on to the next one in case of equality of all preceding ones. Each property name is optionally followed by
+     *                           a column ({@code :}) and an order specifier: {@code asc} or {@code desc}.
+     * @param scrollTimeValidity the time the scrolling query should stay valid. This must contain a time unit value such as the ones supported by ElasticSearch, such as
+     *                           *                     the ones declared here : https://www.elastic.co/guide/en/elasticsearch/reference/current/common-options.html#time-units
      * @return a {@link PartialList} of pages items with the given type
      */
     <T extends Item> PartialList<T> getAllItems(final Class<T> clazz, int offset, int size, String sortBy, String scrollTimeValidity);
@@ -101,10 +101,9 @@ public interface PersistenceService {
     /**
      * Persists the specified Item in the context server.
      *
-     * @param item the item to persist
+     * @param item        the item to persist
      * @param useBatching whether to use batching or not for saving the item. If activating there may be a delay between
-     *                 the call to this method and the actual saving in the persistence backend.
-     *
+     *                    the call to this method and the actual saving in the persistence backend.
      * @return {@code true} if the item was properly persisted, {@code false} otherwise
      */
     boolean save(Item item, boolean useBatching);
@@ -112,11 +111,10 @@ public interface PersistenceService {
     /**
      * Persists the specified Item in the context server.
      *
-     * @param item the item to persist
-     * @param useBatching whether to use batching or not for saving the item. If activating there may be a delay between
-     *                 the call to this method and the actual saving in the persistence backend
+     * @param item            the item to persist
+     * @param useBatching     whether to use batching or not for saving the item. If activating there may be a delay between
+     *                        the call to this method and the actual saving in the persistence backend
      * @param alwaysOverwrite whether to overwrite a document even if we are holding an old item when saving
-     *
      * @return {@code true} if the item was properly persisted, {@code false} otherwise
      */
     boolean save(Item item, Boolean useBatching, Boolean alwaysOverwrite);
@@ -124,87 +122,136 @@ public interface PersistenceService {
     /**
      * Updates the item of the specified class and identified by the specified identifier with new property values provided as name - value pairs in the specified Map.
      *
-     * @param item     the item we want to update
-     * @param dateHint a Date helping in identifying where the item is located
-     * @param clazz    the Item subclass of the item to update
-     * @param source   a Map with entries specifying as key the property name to update and as value its new value
+     * @param item   the item we want to update
+     * @param clazz  the Item subclass of the item to update
+     * @param source a Map with entries specifying as key the property name to update and as value its new value
      * @return {@code true} if the update was successful, {@code false} otherwise
      */
+    default boolean update(Item item, Class<?> clazz, Map<?, ?> source) {
+        return update(item, null, clazz, source);
+    }
+
+    /**
+     * @deprecated use {@link #update(Item, Class, Map)}
+     */
+    @Deprecated
     boolean update(Item item, Date dateHint, Class<?> clazz, Map<?, ?> source);
 
     /**
      * Updates the item of the specified class and identified by the specified identifier with a new property value for the specified property name. Same as
-     * {@code update(itemId, dateHint, clazz, Collections.singletonMap(propertyName, propertyValue))}
+     * {@code update(itemId, clazz, Collections.singletonMap(propertyName, propertyValue))}
      *
      * @param item          the item we want to update
-     * @param dateHint      a Date helping in identifying where the item is located
      * @param clazz         the Item subclass of the item to update
      * @param propertyName  the name of the property to update
      * @param propertyValue the new value of the property
      * @return {@code true} if the update was successful, {@code false} otherwise
      */
+    default boolean update(Item item, Class<?> clazz, String propertyName, Object propertyValue) {
+        return update(item, null, clazz, propertyName, propertyValue);
+    }
+
+    /**
+     * @deprecated use {@link #update(Item, Class, String, Object)}
+     */
+    @Deprecated
     boolean update(Item item, Date dateHint, Class<?> clazz, String propertyName, Object propertyValue);
 
     /**
      * Updates the item of the specified class and identified by the specified identifier with new property values provided as name - value pairs in the specified Map.
      *
-     * @param item     the item we want to update
-     * @param dateHint a Date helping in identifying where the item is located
-     * @param clazz    the Item subclass of the item to update
-     * @param source   a Map with entries specifying as key the property name to update and as value its new value
+     * @param item            the item we want to update
+     * @param clazz           the Item subclass of the item to update
+     * @param source          a Map with entries specifying as key the property name to update and as value its new value
      * @param alwaysOverwrite whether to overwrite a document even if we are holding an old item when saving
      * @return {@code true} if the update was successful, {@code false} otherwise
      */
+    default boolean update(Item item, Class<?> clazz, Map<?, ?> source, final boolean alwaysOverwrite) {
+        return update(item, null, clazz, source, alwaysOverwrite);
+    }
+
+    /**
+     * @deprecated use {@link #update(Item, Class, Map, boolean)}
+     */
+    @Deprecated
     boolean update(Item item, Date dateHint, Class<?> clazz, Map<?, ?> source, final boolean alwaysOverwrite);
 
     /**
      * Updates Map of items of the specified class and identified by the specified identifier with a new property value for the specified property name. Same as
-     * {@code update(itemId, dateHint, clazz, Collections.singletonMap(propertyName, propertyValue))}
+     * {@code update(itemId, clazz, Collections.singletonMap(propertyName, propertyValue))}
      *
-     * @param items         A map the consist of item (key) and properties to update (value)
-     * @param dateHint      a Date helping in identifying where the item is located
-     * @param clazz         the Item subclass of the item to update
+     * @param items A map the consist of item (key) and properties to update (value)
+     * @param clazz the Item subclass of the item to update
      * @return List of failed Items Ids, if all succesful then returns an empty list. if the whole operation failed then will return null
      */
+    default List<String> update(Map<Item, Map> items, Class clazz) {
+        return update(items, null, clazz);
+    }
+
+    /**
+     * @deprecated use {@link #update(Map, Class)}
+     */
+    @Deprecated
     List<String> update(Map<Item, Map> items, Date dateHint, Class clazz);
 
     /**
      * Updates the item of the specified class and identified by the specified identifier with a new property value for the specified property name. Same as
-     * {@code update(itemId, dateHint, clazz, Collections.singletonMap(propertyName, propertyValue))}
+     * {@code update(itemId, clazz, Collections.singletonMap(propertyName, propertyValue))}
      *
-     * @param item          the item we want to update
-     * @param dateHint      a Date helping in identifying where the item is located
-     * @param clazz         the Item subclass of the item to update
-     * @param script        inline script
-     * @param scriptParams  script params
+     * @param item         the item we want to update
+     * @param clazz        the Item subclass of the item to update
+     * @param script       inline script
+     * @param scriptParams script params
      * @return {@code true} if the update was successful, {@code false} otherwise
      */
+    default boolean updateWithScript(Item item, Class<?> clazz, String script, Map<String, Object> scriptParams) {
+        return updateWithScript(item, null, clazz, script, scriptParams);
+    }
+
+    /**
+     * @deprecated use {@link #updateWithScript(Item, Class, String, Map)}
+     */
+    @Deprecated
     boolean updateWithScript(Item item, Date dateHint, Class<?> clazz, String script, Map<String, Object> scriptParams);
 
     /**
      * Updates the items of the specified class by a query with a new property value for the specified property name
      * based on provided scripts and script parameters
      *
-     * @param dateHint      a Date helping in identifying where the item is located
-     * @param clazz         the Item subclass of the item to update
-     * @param scripts       inline scripts array
-     * @param scriptParams  script params array
-     * @param conditions    conditions array
+     * @param clazz        the Item subclass of the item to update
+     * @param scripts      inline scripts array
+     * @param scriptParams script params array
+     * @param conditions   conditions array
      * @return {@code true} if the update was successful, {@code false} otherwise
      */
+    default boolean updateWithQueryAndScript(Class<?> clazz, String[] scripts, Map<String, Object>[] scriptParams, Condition[] conditions) {
+        return updateWithQueryAndScript(null, clazz, scripts, scriptParams, conditions);
+    }
+
+    /**
+     * @deprecated use {@link #updateWithQueryAndScript(Class, String[], Map[], Condition[])}
+     */
+    @Deprecated
     boolean updateWithQueryAndScript(Date dateHint, Class<?> clazz, String[] scripts, Map<String, Object>[] scriptParams, Condition[] conditions);
 
     /**
      * Updates the items of the specified class by a query with a new property value for the specified property name
      * based on provided stored scripts and script parameters
      *
-     * @param dateHint      a Date helping in identifying where the item is located
-     * @param clazz         the Item subclass of the item to update
-     * @param scripts       Stored scripts name
-     * @param scriptParams  script params array
-     * @param conditions    conditions array
+     * @param clazz        the Item subclass of the item to update
+     * @param scripts      Stored scripts name
+     * @param scriptParams script params array
+     * @param conditions   conditions array
      * @return {@code true} if the update was successful, {@code false} otherwise
      */
+    default boolean updateWithQueryAndStoredScript(Class<?> clazz, String[] scripts, Map<String, Object>[] scriptParams, Condition[] conditions) {
+        return updateWithQueryAndStoredScript(null, clazz, scripts, scriptParams, conditions);
+    }
+
+    /**
+     * @deprecated use {@link #updateWithQueryAndStoredScript(Class, String[], Map[], Condition[])}
+     */
+    @Deprecated
     boolean updateWithQueryAndStoredScript(Date dateHint, Class<?> clazz, String[] scripts, Map<String, Object>[] scriptParams, Condition[] conditions);
 
     /**
@@ -226,23 +273,25 @@ public interface PersistenceService {
     <T extends Item> T load(String itemId, Class<T> clazz);
 
     /**
-     * Retrieves the item identified with the specified identifier and with the specified Item subclass if it exists.
-     *
-     * @param <T>      the type of the Item subclass we want to retrieve
-     * @param itemId   the identifier of the item we want to retrieve
-     * @param dateHint a Date helping in identifying where the item is located
-     * @param clazz    the {@link Item} subclass of the item we want to retrieve
-     * @return the item identified with the specified identifier and with the specified Item subclass if it exists, {@code null} otherwise
+     * @deprecated use {@link #load(String, Class)}
      */
+    @Deprecated
     <T extends Item> T load(String itemId, Date dateHint, Class<T> clazz);
 
     /**
      * Load a custom item type identified by an identifier, an optional date hint and the identifier of the custom item type
-     * @param itemId the identifier of the custom type we want to retrieve
-     * @param dateHint an optional Date object if the custom item types are stored by date
+     *
+     * @param itemId         the identifier of the custom type we want to retrieve
      * @param customItemType an identifier of the custom item type to load
      * @return the CustomItem instance with the specified identifier and the custom item type if it exists, {@code null} otherwise
      */
+    default CustomItem loadCustomItem(String itemId, String customItemType) {
+        return loadCustomItem(itemId, null, customItemType);
+    }
+
+    /**
+     * @deprecated use {@link #loadCustomItem(String, String)}
+     */
     CustomItem loadCustomItem(String itemId, Date dateHint, String customItemType);
 
     /**
@@ -257,7 +306,8 @@ public interface PersistenceService {
 
     /**
      * Remove a custom item identified by the custom item identifier and the custom item type identifier
-     * @param itemId the identifier of the custom item to be removed
+     *
+     * @param itemId         the identifier of the custom item to be removed
      * @param customItemType the name of the custom item type
      * @return {@code true} if the deletion was successful, {@code false} otherwise
      */
@@ -293,7 +343,7 @@ public interface PersistenceService {
     /**
      * Retrieve the type mappings for a given itemType. This method queries the persistence service implementation
      * to retrieve any type mappings it may have for the specified itemType.
-     *
+     * <p>
      * This method may not return any results if the implementation doesn't support property type mappings
      *
      * @param itemType the itemType we want to retrieve the mappings for
@@ -320,14 +370,15 @@ public interface PersistenceService {
 
     /**
      * Create mapping
-     * @param type the type
+     *
+     * @param type   the type
      * @param source the source
      */
     void createMapping(String type, String source);
 
     /**
      * Checks whether the specified item satisfies the provided condition.
-     *
+     * <p>
      * TODO: rename to isMatching?
      *
      * @param query the condition we're testing the specified item against
@@ -340,23 +391,24 @@ public interface PersistenceService {
      * validates if a condition throws exception at query build.
      *
      * @param condition the condition we're testing the specified item against
-     * @param item  the item we're checking against the specified condition
+     * @param item      the item we're checking against the specified condition
      * @return {@code true} if the item satisfies the condition, {@code false} otherwise
      */
     boolean isValidCondition(Condition condition, Item item);
+
     /**
      * Same as {@code query(fieldName, fieldValue, sortBy, clazz, 0, -1).getList()}
      *
-     * @see #query(Condition, String, Class, int, int)
-     * @param <T>         the type of the Item subclass we want to retrieve
-     * @param fieldName   the name of the field which we want items to have the specified values
-     * @param fieldValue  the value the items to retrieve should have for the specified field
-     * @param sortBy      an optional ({@code null} if no sorting is required) String of comma ({@code ,}) separated property names on which ordering should be performed, ordering
-     *                    elements according to the property order in the
-     *                    String, considering each in turn and moving on to the next one in case of equality of all preceding ones. Each property name is optionally followed by
-     *                    a column ({@code :}) and an order specifier: {@code asc} or {@code desc}.
-     * @param clazz       the {@link Item} subclass of the items we want to retrieve
+     * @param <T>        the type of the Item subclass we want to retrieve
+     * @param fieldName  the name of the field which we want items to have the specified values
+     * @param fieldValue the value the items to retrieve should have for the specified field
+     * @param sortBy     an optional ({@code null} if no sorting is required) String of comma ({@code ,}) separated property names on which ordering should be performed, ordering
+     *                   elements according to the property order in the
+     *                   String, considering each in turn and moving on to the next one in case of equality of all preceding ones. Each property name is optionally followed by
+     *                   a column ({@code :}) and an order specifier: {@code asc} or {@code desc}.
+     * @param clazz      the {@link Item} subclass of the items we want to retrieve
      * @return a list of items matching the specified criteria
+     * @see #query(Condition, String, Class, int, int)
      */
     <T extends Item> List<T> query(String fieldName, String fieldValue, String sortBy, Class<T> clazz);
 
@@ -465,16 +517,16 @@ public interface PersistenceService {
      * are retrieved, starting with the {@code offset}-th one. If a scroll identifier and time validity are specified, they will be used to perform a scrolling query, meaning
      * that only partial results will be returned, but the scrolling can be continued.
      *
-     * @param <T>    the type of the Item subclass we want to retrieve
-     * @param query  the {@link Condition} the items must satisfy to be retrieved
-     * @param sortBy an optional ({@code null} if no sorting is required) String of comma ({@code ,}) separated property names on which ordering should be performed, ordering
-     *               elements according to the property order in the
-     *               String, considering each in turn and moving on to the next one in case of equality of all preceding ones. Each property name is optionally followed by
-     *               a column ({@code :}) and an order specifier: {@code asc} or {@code desc}.
-     * @param clazz  the {@link Item} subclass of the items we want to retrieve
-     * @param offset zero or a positive integer specifying the position of the first item in the total ordered collection of matching items
-     * @param size   a positive integer specifying how many matching items should be retrieved or {@code -1} if all of them should be retrieved. In the case of a scroll query
-     *               this will be used as the scrolling window size.
+     * @param <T>                the type of the Item subclass we want to retrieve
+     * @param query              the {@link Condition} the items must satisfy to be retrieved
+     * @param sortBy             an optional ({@code null} if no sorting is required) String of comma ({@code ,}) separated property names on which ordering should be performed, ordering
+     *                           elements according to the property order in the
+     *                           String, considering each in turn and moving on to the next one in case of equality of all preceding ones. Each property name is optionally followed by
+     *                           a column ({@code :}) and an order specifier: {@code asc} or {@code desc}.
+     * @param clazz              the {@link Item} subclass of the items we want to retrieve
+     * @param offset             zero or a positive integer specifying the position of the first item in the total ordered collection of matching items
+     * @param size               a positive integer specifying how many matching items should be retrieved or {@code -1} if all of them should be retrieved. In the case of a scroll query
+     *                           this will be used as the scrolling window size.
      * @param scrollTimeValidity the time the scrolling query should stay valid. This must contain a time unit value such as the ones supported by ElasticSearch, such as
      *                           the ones declared here : https://www.elastic.co/guide/en/elasticsearch/reference/current/common-options.html#time-units
      * @return a {@link PartialList} of items matching the specified criteria, with an scroll identifier and the scroll validity used if a scroll query was requested.
@@ -483,11 +535,12 @@ public interface PersistenceService {
 
     /**
      * Continues the execution of a scroll query, to retrieve the next results. If there are no more results the scroll query is also cleared.
-     * @param clazz  the {@link Item} subclass of the items we want to retrieve
-     * @param scrollIdentifier a scroll identifier obtained by the execution of a first query and returned in the {@link PartialList} object
+     *
+     * @param clazz              the {@link Item} subclass of the items we want to retrieve
+     * @param scrollIdentifier   a scroll identifier obtained by the execution of a first query and returned in the {@link PartialList} object
      * @param scrollTimeValidity a scroll time validity value for the scroll query to stay valid. This must contain a time unit value such as the ones supported by ElasticSearch, such as
      *                           the ones declared here : https://www.elastic.co/guide/en/elasticsearch/reference/current/common-options.html#time-units
-     * @param <T>    the type of the Item subclass we want to retrieve
+     * @param <T>                the type of the Item subclass we want to retrieve
      * @return a {@link PartialList} of items matching the specified criteria, with an scroll identifier and the scroll validity used if a scroll query was requested. Note that if
      * there are no more results the list will be empty but not null.
      */
@@ -499,15 +552,15 @@ public interface PersistenceService {
      * {@code offset}-th one. If a scroll identifier and time validity are specified, they will be used to perform a
      * scrolling query, meaning that only partial results will be returned, but the scrolling can be continued.
      *
-     * @param query the {@link Condition} the items must satisfy to be retrieved
-     * @param sortBy an optional ({@code null} if no sorting is required) String of comma ({@code ,}) separated property names on which ordering should be performed, ordering
-     *               elements according to the property order in the
-     *               String, considering each in turn and moving on to the next one in case of equality of all preceding ones. Each property name is optionally followed by
-     *               a column ({@code :}) and an order specifier: {@code asc} or {@code desc}.
-     * @param customItemType the identifier of the custom item type we want to query
-     * @param offset zero or a positive integer specifying the position of the first item in the total ordered collection of matching items
-     * @param size   a positive integer specifying how many matching items should be retrieved or {@code -1} if all of them should be retrieved. In the case of a scroll query
-     *               this will be used as the scrolling window size.
+     * @param query              the {@link Condition} the items must satisfy to be retrieved
+     * @param sortBy             an optional ({@code null} if no sorting is required) String of comma ({@code ,}) separated property names on which ordering should be performed, ordering
+     *                           elements according to the property order in the
+     *                           String, considering each in turn and moving on to the next one in case of equality of all preceding ones. Each property name is optionally followed by
+     *                           a column ({@code :}) and an order specifier: {@code asc} or {@code desc}.
+     * @param customItemType     the identifier of the custom item type we want to query
+     * @param offset             zero or a positive integer specifying the position of the first item in the total ordered collection of matching items
+     * @param size               a positive integer specifying how many matching items should be retrieved or {@code -1} if all of them should be retrieved. In the case of a scroll query
+     *                           this will be used as the scrolling window size.
      * @param scrollTimeValidity the time the scrolling query should stay valid. This must contain a time unit value such as the ones supported by ElasticSearch, such as
      *                           the ones declared here : https://www.elastic.co/guide/en/elasticsearch/reference/current/common-options.html#time-units
      * @return a {@link PartialList} of items matching the specified criteria, with an scroll identifier and the scroll validity used if a scroll query was requested.
@@ -517,8 +570,8 @@ public interface PersistenceService {
     /**
      * Continues the execution of a scroll query, to retrieve the next results. If there are no more results the scroll query is also cleared.
      *
-     * @param customItemType the identifier of the custom item type we want to continue querying
-     * @param scrollIdentifier a scroll identifier obtained by the execution of a first query and returned in the {@link PartialList} object
+     * @param customItemType     the identifier of the custom item type we want to continue querying
+     * @param scrollIdentifier   a scroll identifier obtained by the execution of a first query and returned in the {@link PartialList} object
      * @param scrollTimeValidity a scroll time validity value for the scroll query to stay valid. This must contain a time unit value such as the ones supported by ElasticSearch, such as
      *                           the ones declared here : https://www.elastic.co/guide/en/elasticsearch/reference/current/common-options.html#time-units
      * @return a {@link PartialList} of items matching the specified criteria, with an scroll identifier and the scroll validity used if a scroll query was requested. Note that if
@@ -609,10 +662,17 @@ public interface PersistenceService {
     /**
      * Updates the persistence's engine specific index.
      *
-     * @param clazz will use an index by class type
-     * @param dateHint for index with time, can be null
-     * @param <T> a class that extends Item
+     * @param clazz    will use an index by class type
+     * @param <T>      a class that extends Item
      */
+    default <T extends Item> void refreshIndex(Class<T> clazz) {
+        refreshIndex(clazz, null);
+    }
+
+    /**
+     * @deprecated use {@link #refreshIndex(Class)}
+     */
+    @Deprecated
     <T extends Item> void refreshIndex(Class<T> clazz, Date dateHint);
 
     /**
@@ -655,7 +715,7 @@ public interface PersistenceService {
 
     /**
      * Creates an index with for the specified item type in the persistence engine.
-     *
+     * <p>
      * TODO: remove from API?
      *
      * @param itemType the item type
@@ -665,7 +725,7 @@ public interface PersistenceService {
 
     /**
      * Removes the index for the specified item type.
-     *
+     * <p>
      * TODO: remove from API?
      *
      * @param itemType the item type
diff --git a/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/actions/MergeProfilesOnPropertyAction.java b/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/actions/MergeProfilesOnPropertyAction.java
index 752068594..2261a2a4b 100644
--- a/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/actions/MergeProfilesOnPropertyAction.java
+++ b/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/actions/MergeProfilesOnPropertyAction.java
@@ -162,13 +162,14 @@ public class MergeProfilesOnPropertyAction implements ActionExecutor {
                             // Update current event explicitly, as it might not return from search query if there wasn't a refresh in ES
                             if (!StringUtils.equals(profileId, masterProfileId)) {
                                 if (currentEvent.isPersistent()) {
-                                    persistenceService.update(currentEvent, currentEvent.getTimeStamp(), Event.class, "profileId", anonymousBrowsing ? null : masterProfileId);
+                                    persistenceService.update(currentEvent, Event.class, "profileId", anonymousBrowsing ? null : masterProfileId);
                                 }
                             }
 
                             for (Profile profile : profiles) {
                                 String profileId = profile.getItemId();
                                 if (!StringUtils.equals(profileId, masterProfileId)) {
+                                    // TODO consider udpate by query and/or script
                                     List<Session> sessions = persistenceService.query("profileId", profileId, null, Session.class);
                                     if (currentSession != null) {
                                         if (masterProfileId.equals(profileId) && !sessions.contains(currentSession)) {
@@ -177,13 +178,14 @@ public class MergeProfilesOnPropertyAction implements ActionExecutor {
                                     }
 
                                     for (Session session : sessions) {
-                                        persistenceService.update(session, session.getTimeStamp(), Session.class, "profileId", anonymousBrowsing ? null : masterProfileId);
+                                        persistenceService.update(session, Session.class, "profileId", anonymousBrowsing ? null : masterProfileId);
                                     }
 
+                                    // TODO consider udpate by query and/or script
                                     List<Event> events = persistenceService.query("profileId", profileId, null, Event.class);
                                     for (Event event : events) {
                                         if (!event.getItemId().equals(currentEvent.getItemId())) {
-                                            persistenceService.update(event, event.getTimeStamp(), Event.class, "profileId", anonymousBrowsing ? null : masterProfileId);
+                                            persistenceService.update(event, Event.class, "profileId", anonymousBrowsing ? null : masterProfileId);
                                         }
                                     }
 
diff --git a/plugins/mail/src/main/java/org/apache/unomi/plugins/mail/actions/SendMailAction.java b/plugins/mail/src/main/java/org/apache/unomi/plugins/mail/actions/SendMailAction.java
index acdb1eb7f..3d73e0b32 100644
--- a/plugins/mail/src/main/java/org/apache/unomi/plugins/mail/actions/SendMailAction.java
+++ b/plugins/mail/src/main/java/org/apache/unomi/plugins/mail/actions/SendMailAction.java
@@ -116,7 +116,7 @@ public class SendMailAction implements ActionExecutor {
         event.getProfile().setSystemProperty("notificationAck", profileNotif);
         event.getProfile().setSystemProperty("lastUpdated", new Date());
 
-        persistenceService.update(event.getProfile(), null, Profile.class, "systemProperties", event.getProfile().getSystemProperties());
+        persistenceService.update(event.getProfile(), Profile.class, "systemProperties", event.getProfile().getSystemProperties());
 
         ST stringTemplate = new ST(template, '$', '$');
         stringTemplate.add("profile", event.getProfile());
diff --git a/rest/src/main/java/org/apache/unomi/rest/endpoints/ProfileServiceEndPoint.java b/rest/src/main/java/org/apache/unomi/rest/endpoints/ProfileServiceEndPoint.java
index 86b50b714..e07f36df1 100644
--- a/rest/src/main/java/org/apache/unomi/rest/endpoints/ProfileServiceEndPoint.java
+++ b/rest/src/main/java/org/apache/unomi/rest/endpoints/ProfileServiceEndPoint.java
@@ -391,16 +391,15 @@ public class ProfileServiceEndPoint {
      * Retrieves the session identified by the specified identifier.
      *
      * @param sessionId the identifier of the session to be retrieved
-     * @param dateHint  a Date helping in identifying where the item is located
      * @return the session identified by the specified identifier
      * @throws ParseException if the date hint cannot be parsed as a proper {@link Date} object
      */
     @GET
     @Path("/sessions/{sessionId}")
-    public Session loadSession(@PathParam("sessionId") String sessionId, @QueryParam("dateHint") String dateHint) throws ParseException {
-        return profileService.loadSession(sessionId, dateHint != null ? new SimpleDateFormat("yyyy-MM").parse(dateHint) : null);
+    public Session loadSession(@PathParam("sessionId") String sessionId) throws ParseException {
+        return profileService.loadSession(sessionId);
     }
-
+    
     /**
      * Saves the specified session.
      *
diff --git a/rest/src/main/java/org/apache/unomi/rest/service/impl/RestServiceUtilsImpl.java b/rest/src/main/java/org/apache/unomi/rest/service/impl/RestServiceUtilsImpl.java
index ace630b3f..4d0e10a17 100644
--- a/rest/src/main/java/org/apache/unomi/rest/service/impl/RestServiceUtilsImpl.java
+++ b/rest/src/main/java/org/apache/unomi/rest/service/impl/RestServiceUtilsImpl.java
@@ -137,7 +137,7 @@ public class RestServiceUtilsImpl implements RestServiceUtils {
             Profile sessionProfile;
             if (StringUtils.isNotBlank(sessionId) && !invalidateSession) {
 
-                eventsRequestContext.setSession(profileService.loadSession(sessionId, timestamp));
+                eventsRequestContext.setSession(profileService.loadSession(sessionId));
                 if (eventsRequestContext.getSession() != null) {
 
                     sessionProfile = eventsRequestContext.getSession().getProfile();
diff --git a/services/src/main/java/org/apache/unomi/services/impl/profiles/ProfileServiceImpl.java b/services/src/main/java/org/apache/unomi/services/impl/profiles/ProfileServiceImpl.java
index bc93fd0c5..c12dc411f 100644
--- a/services/src/main/java/org/apache/unomi/services/impl/profiles/ProfileServiceImpl.java
+++ b/services/src/main/java/org/apache/unomi/services/impl/profiles/ProfileServiceImpl.java
@@ -144,6 +144,7 @@ public class ProfileServiceImpl implements ProfileService, SynchronousBundleList
         /**
          * Creates a new instance of this class containing given property types.
          * If property types with the same ID existed before, they will be replaced by the new ones.
+         *
          * @param newProperties list of property types to change
          * @return new instance
          */
@@ -165,13 +166,14 @@ public class ProfileServiceImpl implements ProfileService, SynchronousBundleList
 
         /**
          * Creates a new instance of this class containing all property types except the one with given ID.
+         *
          * @param propertyId ID of the property to delete
          * @return new instance
          */
         public PropertyTypes without(String propertyId) {
             List<PropertyType> newPropertyTypes = allPropertyTypes.stream()
-                .filter(property -> !property.getItemId().equals(propertyId))
-                .collect(Collectors.toList());
+                    .filter(property -> !property.getItemId().equals(propertyId))
+                    .collect(Collectors.toList());
 
             return new PropertyTypes(newPropertyTypes);
         }
@@ -847,17 +849,14 @@ public class ProfileServiceImpl implements ProfileService, SynchronousBundleList
         return null;
     }
 
+    @Override
     public Session loadSession(String sessionId, Date dateHint) {
-        Session s = persistenceService.load(sessionId, dateHint, Session.class);
-        if (s == null && dateHint != null) {
-            GregorianCalendar gc = new GregorianCalendar();
-            gc.setTime(dateHint);
-            if (gc.get(Calendar.DAY_OF_MONTH) == 1) {
-                gc.add(Calendar.DAY_OF_MONTH, -1);
-                s = persistenceService.load(sessionId, gc.getTime(), Session.class);
-            }
-        }
-        return s;
+        return loadSession(sessionId);
+    }
+
+    @Override
+    public Session loadSession(String sessionId) {
+        return persistenceService.load(sessionId, Session.class);
     }
 
     public Session saveSession(Session session) {
diff --git a/services/src/main/java/org/apache/unomi/services/impl/segments/SegmentServiceImpl.java b/services/src/main/java/org/apache/unomi/services/impl/segments/SegmentServiceImpl.java
index 56f1287af..1b64cbdc5 100644
--- a/services/src/main/java/org/apache/unomi/services/impl/segments/SegmentServiceImpl.java
+++ b/services/src/main/java/org/apache/unomi/services/impl/segments/SegmentServiceImpl.java
@@ -746,7 +746,7 @@ public class SegmentServiceImpl extends AbstractServiceImpl implements SegmentSe
                 // todo remove profile properties ?
                 persistenceService.remove(previousRule.getItemId(), Rule.class);
             } else {
-                persistenceService.update(previousRule, null, Rule.class, "linkedItems", previousRule.getLinkedItems());
+                persistenceService.update(previousRule, Rule.class, "linkedItems", previousRule.getLinkedItems());
             }
         }
     }
@@ -1094,7 +1094,7 @@ public class SegmentServiceImpl extends AbstractServiceImpl implements SegmentSe
             } else { //send update profile one by one
                 for (Profile profileToUpdate : profiles.getList()) {
                     Map<String, Object> sourceMap = buildPropertiesMapForUpdateSegment(profileToUpdate, segmentId, isAdd);
-                    persistenceService.update(profileToUpdate, null, Profile.class, sourceMap);
+                    persistenceService.update(profileToUpdate, Profile.class, sourceMap);
                 }
             }
             if (sendProfileUpdateEvent)
@@ -1131,7 +1131,7 @@ public class SegmentServiceImpl extends AbstractServiceImpl implements SegmentSe
                         logger.warn("retry updating profile segment {}, profile {}, time {}", segmentId, profileId, new Date());
                         Profile profileToAddUpdated = persistenceService.load(profileId, Profile.class);
                         Map<String, Object> sourceMapToUpdate = buildPropertiesMapForUpdateSegment(profileToAddUpdated, segmentId, isAdd);
-                        boolean isUpdated = persistenceService.update(profileToAddUpdated, null, Profile.class, sourceMapToUpdate);
+                        boolean isUpdated = persistenceService.update(profileToAddUpdated, Profile.class, sourceMapToUpdate);
                         if (isUpdated == false)
                             throw new Exception(String.format("failed retry update profile segment {}, profile {}, time {}", segmentId, profileId, new Date()));
                     });
@@ -1191,7 +1191,7 @@ public class SegmentServiceImpl extends AbstractServiceImpl implements SegmentSe
                 idx++;
             }
         }
-        persistenceService.updateWithQueryAndStoredScript(null, Profile.class, scripts, scriptParams, conditions);
+        persistenceService.updateWithQueryAndStoredScript(Profile.class, scripts, scriptParams, conditions);
         logger.info("Updated scoring for profiles in {}ms", System.currentTimeMillis() - startTime);
     }
 
diff --git a/tools/shell-dev-commands/src/main/java/org/apache/unomi/shell/commands/SessionView.java b/tools/shell-dev-commands/src/main/java/org/apache/unomi/shell/commands/SessionView.java
index d8d96ade1..7d5f6846d 100644
--- a/tools/shell-dev-commands/src/main/java/org/apache/unomi/shell/commands/SessionView.java
+++ b/tools/shell-dev-commands/src/main/java/org/apache/unomi/shell/commands/SessionView.java
@@ -36,7 +36,7 @@ public class SessionView implements Action {
     String sessionIdentifier;
 
     public Object execute() throws Exception {
-        Session session = profileService.loadSession(sessionIdentifier, null);
+        Session session = profileService.loadSession(sessionIdentifier);
         if (session == null) {
             System.out.println("Couldn't find a session with id=" + sessionIdentifier);
             return null;