You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@unomi.apache.org by dg...@apache.org on 2018/06/29 08:32:49 UTC

[1/3] incubator-unomi git commit: UNOMI-185 create an additional endpoint to execute aggregate query, the new endpoint return a bit less information but is much faster

Repository: incubator-unomi
Updated Branches:
  refs/heads/unomi-1.3.x af7bfb114 -> b71269408


UNOMI-185 create an additional endpoint to execute aggregate query, the new endpoint return a bit less information but is much faster


Project: http://git-wip-us.apache.org/repos/asf/incubator-unomi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-unomi/commit/7129cdb3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-unomi/tree/7129cdb3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-unomi/diff/7129cdb3

Branch: refs/heads/unomi-1.3.x
Commit: 7129cdb35e1624de3fa2e29eb2f7f2c19ee3f1ec
Parents: af7bfb1
Author: dgaillard <dg...@jahia.com>
Authored: Tue Jun 26 16:36:09 2018 +0200
Committer: dgaillard <dg...@jahia.com>
Committed: Fri Jun 29 10:31:36 2018 +0200

----------------------------------------------------------------------
 .../apache/unomi/api/services/QueryService.java | 14 +++++
 .../ElasticSearchPersistenceServiceImpl.java    | 64 ++++++++++++++------
 .../persistence/spi/PersistenceService.java     | 13 ++++
 .../BooleanConditionESQueryBuilder.java         | 12 +++-
 .../apache/unomi/rest/QueryServiceEndPoint.java | 19 ++++++
 .../services/services/QueryServiceImpl.java     | 61 ++++++++++++-------
 6 files changed, 140 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/7129cdb3/api/src/main/java/org/apache/unomi/api/services/QueryService.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/org/apache/unomi/api/services/QueryService.java b/api/src/main/java/org/apache/unomi/api/services/QueryService.java
index 2465e65..3f3cb7b 100644
--- a/api/src/main/java/org/apache/unomi/api/services/QueryService.java
+++ b/api/src/main/java/org/apache/unomi/api/services/QueryService.java
@@ -44,6 +44,7 @@ public interface QueryService {
      *
      * Retrieves the number of items with the specified type as defined by the Item subclass public field {@code ITEM_TYPE} and aggregated by possible values of the specified
      * property or, if the specified query is not {@code null}, perform that aggregate query.
+     * Also return the global count of document matching the {@code ITEM_TYPE}
      *
      * @param itemType the String representation of the item type we want to retrieve the count of, as defined by its class' {@code ITEM_TYPE} field
      * @param property the property we're aggregating on, i.e. for each possible value of this property, we are counting how many items of the specified type have that value
@@ -54,6 +55,19 @@ public interface QueryService {
     Map<String, Long> getAggregate(String itemType, String property, AggregateQuery query);
 
     /**
+     * Retrieves the number of items with the specified type as defined by the Item subclass public field {@code ITEM_TYPE} and aggregated by possible values of the specified
+     * property or, if the specified query is not {@code null}, perform that aggregate query.
+     * This aggregate won't return the global count and should therefore be much faster than {@link #getAggregate(String, String, AggregateQuery)}
+     *
+     * @param itemType the String representation of the item type we want to retrieve the count of, as defined by its class' {@code ITEM_TYPE} field
+     * @param property the property we're aggregating on, i.e. for each possible value of this property, we are counting how many items of the specified type have that value
+     * @param query    the {@link AggregateQuery} specifying the aggregation that should be perfomed
+     * @return a Map associating a specific value of the property to the cardinality of items with that value
+     * @see Item Item for a discussion of {@code ITEM_TYPE}
+     */
+    Map<String, Long> getAggregateOptimized(String itemType, String property, AggregateQuery query);
+
+    /**
      * Retrieves the number of items of the specified type as defined by the Item subclass public field {@code ITEM_TYPE} and matching the specified {@link Condition}.
      *
      * @param condition the condition the items must satisfy

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/7129cdb3/persistence-elasticsearch/core/src/main/java/org/apache/unomi/persistence/elasticsearch/ElasticSearchPersistenceServiceImpl.java
----------------------------------------------------------------------
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 d5fa185..c7ec8d0 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
@@ -1483,6 +1483,16 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
 
     @Override
     public Map<String, Long> aggregateQuery(final Condition filter, final BaseAggregate aggregate, final String itemType) {
+        return aggregateQuery(filter, aggregate, itemType, false);
+    }
+
+    @Override
+    public Map<String, Long> aggregateQueryOptimized(Condition filter, BaseAggregate aggregate, String itemType) {
+        return aggregateQuery(filter, aggregate, itemType, true);
+    }
+
+    private Map<String, Long> aggregateQuery(final Condition filter, final BaseAggregate aggregate, final String itemType, final boolean
+            optimized) {
         return new InClassLoaderExecute<Map<String, Long>>(metricsService, this.getClass().getName() + ".aggregateQuery") {
 
             @Override
@@ -1560,34 +1570,50 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                     }
                 }
 
-                if (filter != null) {
-                    AggregationBuilder filterAggregation = AggregationBuilders.filter("filter", conditionESQueryBuilderDispatcher.buildFilter(filter));
+                // If the request is optimized then we don't need a global aggregation which is very slow and we can put the query with a
+                // filter on range items in the query block so we don't retrieve all the document before filtering the whole
+                if (optimized) {
                     for (AggregationBuilder aggregationBuilder : lastAggregation) {
-                        filterAggregation.subAggregation(aggregationBuilder);
+                        builder.addAggregation(aggregationBuilder);
                     }
-                    lastAggregation = Collections.singletonList(filterAggregation);
-                }
 
+                    if (filter != null) {
+                        builder.setQuery(conditionESQueryBuilderDispatcher.buildFilter(filter));
+                    }
+                } else {
+                    if (filter != null) {
+                        AggregationBuilder filterAggregation = AggregationBuilders.filter("filter", conditionESQueryBuilderDispatcher.buildFilter(filter));
+                        for (AggregationBuilder aggregationBuilder : lastAggregation) {
+                            filterAggregation.subAggregation(aggregationBuilder);
+                        }
+                        lastAggregation = Collections.singletonList(filterAggregation);
+                    }
 
-                AggregationBuilder globalAggregation = AggregationBuilders.global("global");
-                for (AggregationBuilder aggregationBuilder : lastAggregation) {
-                    globalAggregation.subAggregation(aggregationBuilder);
-                }
+                    AggregationBuilder globalAggregation = AggregationBuilders.global("global");
+                    for (AggregationBuilder aggregationBuilder : lastAggregation) {
+                        globalAggregation.subAggregation(aggregationBuilder);
+                    }
 
-                builder.addAggregation(globalAggregation);
+                    builder.addAggregation(globalAggregation);
+                }
 
                 SearchResponse response = builder.execute().actionGet();
-
                 Aggregations aggregations = response.getAggregations();
                 if (aggregations != null) {
-                    Global globalAgg = aggregations.get("global");
-                    results.put("_all", globalAgg.getDocCount());
-                    aggregations = globalAgg.getAggregations();
-
-                    if (aggregations.get("filter") != null) {
-                        Filter filterAgg = aggregations.get("filter");
-                        results.put("_filtered", filterAgg.getDocCount());
-                        aggregations = filterAgg.getAggregations();
+                    if (optimized) {
+                        if (response.getHits() != null) {
+                            results.put("_filtered", response.getHits().getTotalHits());
+                        }
+                    } else {
+                        Global globalAgg = aggregations.get("global");
+                        results.put("_all", globalAgg.getDocCount());
+                        aggregations = globalAgg.getAggregations();
+
+                        if (aggregations.get("filter") != null) {
+                            Filter filterAgg = aggregations.get("filter");
+                            results.put("_filtered", filterAgg.getDocCount());
+                            aggregations = filterAgg.getAggregations();
+                        }
                     }
                     if (aggregations.get("buckets") != null) {
                         MultiBucketsAggregation terms = aggregations.get("buckets");

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/7129cdb3/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/PersistenceService.java
----------------------------------------------------------------------
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 b38531b..c98eb17 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
@@ -413,6 +413,7 @@ public interface PersistenceService {
     /**
      * Retrieves the number of items with the specified type as defined by the Item subclass public field {@code ITEM_TYPE} matching the optional specified condition and
      * aggregated according to the specified {@link BaseAggregate}.
+     * Also return the global count of document matching the {@code ITEM_TYPE}
      *
      * @param filter    the condition the items must match or {@code null} if no filtering is needed
      * @param aggregate an aggregate specifying how matching items must be bundled
@@ -422,6 +423,18 @@ public interface PersistenceService {
     Map<String, Long> aggregateQuery(Condition filter, BaseAggregate aggregate, String itemType);
 
     /**
+     * Retrieves the number of items with the specified type as defined by the Item subclass public field {@code ITEM_TYPE} matching the optional specified condition and
+     * aggregated according to the specified {@link BaseAggregate}.
+     * This aggregate won't return the global count and should therefore be much faster than {@link #aggregateQuery(Condition, BaseAggregate, String)}
+     *
+     * @param filter    the condition the items must match or {@code null} if no filtering is needed
+     * @param aggregate an aggregate specifying how matching items must be bundled
+     * @param itemType  the String representation of the item type we want to retrieve the count of, as defined by its class' {@code ITEM_TYPE} field
+     * @return a Map associating aggregation dimension name as key and cardinality for that dimension as value
+     */
+    Map<String, Long> aggregateQueryOptimized(Condition filter, BaseAggregate aggregate, String itemType);
+
+    /**
      * Updates the persistence's engine indices if needed.
      */
     void refresh();

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/7129cdb3/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/BooleanConditionESQueryBuilder.java
----------------------------------------------------------------------
diff --git a/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/BooleanConditionESQueryBuilder.java b/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/BooleanConditionESQueryBuilder.java
index 7f85a1a..8a7a381 100644
--- a/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/BooleanConditionESQueryBuilder.java
+++ b/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/BooleanConditionESQueryBuilder.java
@@ -54,14 +54,22 @@ public class BooleanConditionESQueryBuilder implements ConditionESQueryBuilder {
             if (isAndOperator) {
                 QueryBuilder andFilter = dispatcher.buildFilter(conditions.get(i), context);
                 if (andFilter != null) {
-                    boolQueryBuilder.must(andFilter);
+                    if (andFilter.getName().equals("range")) {
+                        boolQueryBuilder.filter(andFilter);
+                    } else {
+                        boolQueryBuilder.must(andFilter);
+                    }
                 } else {
                     logger.warn("Null filter for boolean AND sub condition " + conditions.get(i));
                 }
             } else {
                 QueryBuilder orFilter = dispatcher.buildFilter(conditions.get(i), context);
                 if (orFilter != null) {
-                    boolQueryBuilder.should(orFilter);
+                    if (orFilter.getName().equals("range")) {
+                        boolQueryBuilder.filter(orFilter);
+                    } else {
+                        boolQueryBuilder.should(orFilter);
+                    }
                 } else {
                     logger.warn("Null filter for boolean OR sub condition " + conditions.get(i));
                 }

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/7129cdb3/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java
----------------------------------------------------------------------
diff --git a/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java b/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java
index 33c6bdb..3d68e05 100644
--- a/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java
+++ b/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java
@@ -82,6 +82,7 @@ public class QueryServiceEndPoint {
      *
      * Retrieves the number of items with the specified type as defined by the Item subclass public field {@code ITEM_TYPE} and aggregated by possible values of the specified
      * property or, if the specified query is not {@code null}, perform that aggregate query.
+     * Also return the global count of document matching the {@code ITEM_TYPE}
      *
      * @param type           the String representation of the item type we want to retrieve the count of, as defined by its class' {@code ITEM_TYPE} field
      * @param property       the property we're aggregating on, i.e. for each possible value of this property, we are counting how many items of the specified type have that value
@@ -96,6 +97,24 @@ public class QueryServiceEndPoint {
     }
 
     /**
+     * Retrieves the number of items with the specified type as defined by the Item subclass public field {@code ITEM_TYPE} and aggregated by possible values of the specified
+     * property or, if the specified query is not {@code null}, perform that aggregate query.
+     * This aggregate won't return the global count and should therefore be much faster than {@link #getAggregate(String, String, AggregateQuery)}
+     *
+     * @param type           the String representation of the item type we want to retrieve the count of, as defined by its class' {@code ITEM_TYPE} field
+     * @param property       the property we're aggregating on, i.e. for each possible value of this property, we are counting how many items of the specified type have that value
+     * @param aggregateQuery the {@link AggregateQuery} specifying the aggregation that should be perfomed
+     * @return a Map associating a specific value of the property to the cardinality of items with that value
+     * @see Item Item for a discussion of {@code ITEM_TYPE}
+     */
+    @POST
+    @Path("/{type}/{property}/optimized")
+    public Map<String, Long> getAggregateOptimized(@PathParam("type") String type, @PathParam("property") String property, AggregateQuery
+            aggregateQuery) {
+        return queryService.getAggregateOptimized(type, property, aggregateQuery);
+    }
+
+    /**
      * Retrieves the specified metrics for the specified field of items of the specified type as defined by the Item subclass public field {@code ITEM_TYPE} and matching the
      * specified {@link Condition}.
      *

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/7129cdb3/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java
----------------------------------------------------------------------
diff --git a/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java b/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java
index bb9de1d..5da7076 100644
--- a/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java
+++ b/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java
@@ -58,6 +58,31 @@ public class QueryServiceImpl implements QueryService {
 
     @Override
     public Map<String, Long> getAggregate(String itemType, String property, AggregateQuery query) {
+        return getAggregate(itemType, property, query, false);
+    }
+
+    @Override
+    public Map<String, Long> getAggregateOptimized(String itemType, String property, AggregateQuery query) {
+        return getAggregate(itemType, property, query, true);
+    }
+
+    @Override
+    public Map<String, Double> getMetric(String type, String property, String slashConcatenatedMetrics, Condition condition) {
+        if (condition.getConditionType() == null) {
+            ParserHelper.resolveConditionType(definitionsService, condition);
+        }
+        return persistenceService.getSingleValuesMetrics(condition, slashConcatenatedMetrics.split("/"), property, type);
+    }
+
+    @Override
+    public long getQueryCount(String itemType, Condition condition) {
+        if (condition.getConditionType() == null) {
+            ParserHelper.resolveConditionType(definitionsService, condition);
+        }
+        return persistenceService.queryCount(condition, itemType);
+    }
+
+    private Map<String, Long> getAggregate(String itemType, String property, AggregateQuery query, boolean optimized) {
         if (query != null) {
             // resolve condition
             if (query.getCondition() != null) {
@@ -65,6 +90,7 @@ public class QueryServiceImpl implements QueryService {
             }
 
             // resolve aggregate
+            BaseAggregate baseAggregate = null;
             if (query.getAggregate() != null) {
                 String aggregateType = query.getAggregate().getType();
                 if (aggregateType != null) {
@@ -72,40 +98,31 @@ public class QueryServiceImpl implements QueryService {
                     if (aggregateType.equals("date")) {
                         String interval = (String) query.getAggregate().getParameters().get("interval");
                         String format = (String) query.getAggregate().getParameters().get("format");
-                        return persistenceService.aggregateQuery(query.getCondition(), new DateAggregate(property, interval, format), itemType);
+                        baseAggregate = new DateAggregate(property, interval, format);
                     } else if (aggregateType.equals("dateRange") && query.getAggregate().getDateRanges() != null && query.getAggregate().getDateRanges().size() > 0) {
                         String format = (String) query.getAggregate().getParameters().get("format");
-                        return persistenceService.aggregateQuery(query.getCondition(), new DateRangeAggregate(query.getAggregate().getProperty(), format, query.getAggregate().getDateRanges()), itemType);
+                        baseAggregate = new DateRangeAggregate(query.getAggregate().getProperty(), format, query.getAggregate().getDateRanges());
                     } else if (aggregateType.equals("numericRange") && query.getAggregate().getNumericRanges() != null && query.getAggregate().getNumericRanges().size() > 0) {
-                        return persistenceService.aggregateQuery(query.getCondition(), new NumericRangeAggregate(query.getAggregate().getProperty(), query.getAggregate().getNumericRanges()), itemType);
+                        baseAggregate = new NumericRangeAggregate(query.getAggregate().getProperty(), query.getAggregate().getNumericRanges());
                     } else if (aggregateType.equals("ipRange") && query.getAggregate().ipRanges() != null && query.getAggregate().ipRanges().size() > 0) {
-                        return persistenceService.aggregateQuery(query.getCondition(), new IpRangeAggregate(query.getAggregate().getProperty(), query.getAggregate().ipRanges()), itemType);
+                        baseAggregate = new IpRangeAggregate(query.getAggregate().getProperty(), query.getAggregate().ipRanges());
                     }
                 }
             }
 
+            if (baseAggregate == null) {
+                baseAggregate = new TermsAggregate(property);
+            }
+
             // fall back on terms aggregate
-            return persistenceService.aggregateQuery(query.getCondition(), new TermsAggregate(property), itemType);
+            if (optimized) {
+                return persistenceService.aggregateQueryOptimized(query.getCondition(), baseAggregate, itemType);
+            } else {
+                return persistenceService.aggregateQuery(query.getCondition(), baseAggregate, itemType);
+            }
         }
 
         return getAggregate(itemType, property);
     }
 
-    @Override
-    public Map<String, Double> getMetric(String type, String property, String slashConcatenatedMetrics, Condition condition) {
-        if (condition.getConditionType() == null) {
-            ParserHelper.resolveConditionType(definitionsService, condition);
-        }
-        return persistenceService.getSingleValuesMetrics(condition, slashConcatenatedMetrics.split("/"), property, type);
-    }
-
-    @Override
-    public long getQueryCount(String itemType, Condition condition) {
-        if (condition.getConditionType() == null) {
-            ParserHelper.resolveConditionType(definitionsService, condition);
-        }
-        return persistenceService.queryCount(condition, itemType);
-    }
-
-
 }
\ No newline at end of file


[3/3] incubator-unomi git commit: UNOMI-185 use the optimized aggregation when possible

Posted by dg...@apache.org.
UNOMI-185 use the optimized aggregation when possible


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

Branch: refs/heads/unomi-1.3.x
Commit: b712694083ae34cf175958043388f709143a2741
Parents: f579e14
Author: dgaillard <dg...@jahia.com>
Authored: Wed Jun 27 18:47:12 2018 +0200
Committer: dgaillard <dg...@jahia.com>
Committed: Fri Jun 29 10:32:28 2018 +0200

----------------------------------------------------------------------
 .../org/apache/unomi/privacy/internal/PrivacyServiceImpl.java    | 2 +-
 .../baseplugin/conditions/PastEventConditionESQueryBuilder.java  | 2 +-
 .../org/apache/unomi/services/services/EventServiceImpl.java     | 2 +-
 .../org/apache/unomi/services/services/GoalsServiceImpl.java     | 4 ++--
 .../org/apache/unomi/services/services/SegmentServiceImpl.java   | 2 +-
 5 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/b7126940/extensions/privacy-extension/services/src/main/java/org/apache/unomi/privacy/internal/PrivacyServiceImpl.java
----------------------------------------------------------------------
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 d4aad1f..247a7f4 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
@@ -68,7 +68,7 @@ public class PrivacyServiceImpl implements PrivacyService {
         serverInfo.setServerVersion(bundleContext.getBundle().getVersion().toString());
 
         // let's retrieve all the event types the server has seen.
-        Map<String, Long> eventTypeCounts = persistenceService.aggregateQuery(null, new TermsAggregate("eventType"), Event.ITEM_TYPE);
+        Map<String, Long> eventTypeCounts = persistenceService.aggregateWithOptimizedQuery(null, new TermsAggregate("eventType"), Event.ITEM_TYPE);
         List<EventInfo> eventTypes = new ArrayList<EventInfo>();
         for (Map.Entry<String, Long> eventTypeEntry : eventTypeCounts.entrySet()) {
             EventInfo eventInfo = new EventInfo();

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/b7126940/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/PastEventConditionESQueryBuilder.java
----------------------------------------------------------------------
diff --git a/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/PastEventConditionESQueryBuilder.java b/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/PastEventConditionESQueryBuilder.java
index 7c6217b..e51aaa8 100644
--- a/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/PastEventConditionESQueryBuilder.java
+++ b/plugins/baseplugin/src/main/java/org/apache/unomi/plugins/baseplugin/conditions/PastEventConditionESQueryBuilder.java
@@ -75,7 +75,7 @@ public class PastEventConditionESQueryBuilder implements ConditionESQueryBuilder
         Integer minimumEventCount = condition.getParameter("minimumEventCount") == null ? 0 : (Integer) condition.getParameter("minimumEventCount");
         Integer maximumEventCount = condition.getParameter("maximumEventCount") == null  ? Integer.MAX_VALUE : (Integer) condition.getParameter("maximumEventCount");
 
-        Map<String, Long> eventCountByProfile = persistenceService.aggregateQuery(andCondition, new TermsAggregate("profileId"), Event.ITEM_TYPE);
+        Map<String, Long> eventCountByProfile = persistenceService.aggregateWithOptimizedQuery(andCondition, new TermsAggregate("profileId"), Event.ITEM_TYPE);
         if (eventCountByProfile != null) {
             for (Map.Entry<String, Long> entry : eventCountByProfile.entrySet()) {
                 if (!entry.getKey().startsWith("_")) {

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/b7126940/services/src/main/java/org/apache/unomi/services/services/EventServiceImpl.java
----------------------------------------------------------------------
diff --git a/services/src/main/java/org/apache/unomi/services/services/EventServiceImpl.java b/services/src/main/java/org/apache/unomi/services/services/EventServiceImpl.java
index 54b85e4..5e1ed66 100644
--- a/services/src/main/java/org/apache/unomi/services/services/EventServiceImpl.java
+++ b/services/src/main/java/org/apache/unomi/services/services/EventServiceImpl.java
@@ -195,7 +195,7 @@ public class EventServiceImpl implements EventService {
     }
 
     public Set<String> getEventTypeIds() {
-        Map<String, Long> dynamicEventTypeIds = persistenceService.aggregateQuery(null, new TermsAggregate("eventType"), Event.ITEM_TYPE);
+        Map<String, Long> dynamicEventTypeIds = persistenceService.aggregateWithOptimizedQuery(null, new TermsAggregate("eventType"), Event.ITEM_TYPE);
         Set<String> eventTypeIds = new LinkedHashSet<String>(predefinedEventTypeIds);
         eventTypeIds.addAll(dynamicEventTypeIds.keySet());
         return eventTypeIds;

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/b7126940/services/src/main/java/org/apache/unomi/services/services/GoalsServiceImpl.java
----------------------------------------------------------------------
diff --git a/services/src/main/java/org/apache/unomi/services/services/GoalsServiceImpl.java b/services/src/main/java/org/apache/unomi/services/services/GoalsServiceImpl.java
index bd7fce5..771e21f 100644
--- a/services/src/main/java/org/apache/unomi/services/services/GoalsServiceImpl.java
+++ b/services/src/main/java/org/apache/unomi/services/services/GoalsServiceImpl.java
@@ -491,11 +491,11 @@ public class GoalsServiceImpl implements GoalsService, SynchronousBundleListener
 
         if (aggregate != null) {
             list.add(goalStartCondition);
-            all = persistenceService.aggregateQuery(condition, aggregate, Session.ITEM_TYPE);
+            all = persistenceService.aggregateWithOptimizedQuery(condition, aggregate, Session.ITEM_TYPE);
 
             list.remove(goalStartCondition);
             list.add(goalTargetCondition);
-            match = persistenceService.aggregateQuery(condition, aggregate, Session.ITEM_TYPE);
+            match = persistenceService.aggregateWithOptimizedQuery(condition, aggregate, Session.ITEM_TYPE);
         } else {
             list.add(goalStartCondition);
             all = new HashMap<String, Long>();

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/b7126940/services/src/main/java/org/apache/unomi/services/services/SegmentServiceImpl.java
----------------------------------------------------------------------
diff --git a/services/src/main/java/org/apache/unomi/services/services/SegmentServiceImpl.java b/services/src/main/java/org/apache/unomi/services/services/SegmentServiceImpl.java
index a3dc984..3d1067b 100644
--- a/services/src/main/java/org/apache/unomi/services/services/SegmentServiceImpl.java
+++ b/services/src/main/java/org/apache/unomi/services/services/SegmentServiceImpl.java
@@ -768,7 +768,7 @@ public class SegmentServiceImpl extends AbstractServiceImpl implements SegmentSe
             l.add(numberOfDaysCondition);
         }
         String propertyKey = (String) parentCondition.getParameter("generatedPropertyKey");
-        Map<String, Long> eventCountByProfile = persistenceService.aggregateQuery(andCondition, new TermsAggregate("profileId"), Event.ITEM_TYPE);
+        Map<String, Long> eventCountByProfile = persistenceService.aggregateWithOptimizedQuery(andCondition, new TermsAggregate("profileId"), Event.ITEM_TYPE);
         for (Map.Entry<String, Long> entry : eventCountByProfile.entrySet()) {
             String profileId = entry.getKey();
             if (!profileId.startsWith("_")) {


[2/3] incubator-unomi git commit: UNOMI-185 refactor and clean code after successful test with optimized query

Posted by dg...@apache.org.
UNOMI-185 refactor and clean code after successful test with optimized query


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

Branch: refs/heads/unomi-1.3.x
Commit: f579e1423c51bfb4a1b174a77ce3126ac2185867
Parents: 7129cdb
Author: dgaillard <dg...@jahia.com>
Authored: Wed Jun 27 12:01:56 2018 +0200
Committer: dgaillard <dg...@jahia.com>
Committed: Fri Jun 29 10:31:59 2018 +0200

----------------------------------------------------------------------
 .../apache/unomi/api/services/QueryService.java |  2 +-
 .../ElasticSearchPersistenceServiceImpl.java    | 12 +++----
 .../persistence/spi/PersistenceService.java     |  2 +-
 .../apache/unomi/rest/QueryServiceEndPoint.java | 33 +++++++-------------
 .../services/services/QueryServiceImpl.java     |  8 ++---
 5 files changed, 23 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/f579e142/api/src/main/java/org/apache/unomi/api/services/QueryService.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/org/apache/unomi/api/services/QueryService.java b/api/src/main/java/org/apache/unomi/api/services/QueryService.java
index 3f3cb7b..6ff5cc3 100644
--- a/api/src/main/java/org/apache/unomi/api/services/QueryService.java
+++ b/api/src/main/java/org/apache/unomi/api/services/QueryService.java
@@ -65,7 +65,7 @@ public interface QueryService {
      * @return a Map associating a specific value of the property to the cardinality of items with that value
      * @see Item Item for a discussion of {@code ITEM_TYPE}
      */
-    Map<String, Long> getAggregateOptimized(String itemType, String property, AggregateQuery query);
+    Map<String, Long> getAggregateWithOptimizedQuery(String itemType, String property, AggregateQuery query);
 
     /**
      * Retrieves the number of items of the specified type as defined by the Item subclass public field {@code ITEM_TYPE} and matching the specified {@link Condition}.

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/f579e142/persistence-elasticsearch/core/src/main/java/org/apache/unomi/persistence/elasticsearch/ElasticSearchPersistenceServiceImpl.java
----------------------------------------------------------------------
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 c7ec8d0..0f527db 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
@@ -1482,17 +1482,17 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
     }
 
     @Override
-    public Map<String, Long> aggregateQuery(final Condition filter, final BaseAggregate aggregate, final String itemType) {
+    public Map<String, Long> aggregateQuery(Condition filter, BaseAggregate aggregate, String itemType) {
         return aggregateQuery(filter, aggregate, itemType, false);
     }
 
     @Override
-    public Map<String, Long> aggregateQueryOptimized(Condition filter, BaseAggregate aggregate, String itemType) {
+    public Map<String, Long> aggregateWithOptimizedQuery(Condition filter, BaseAggregate aggregate, String itemType) {
         return aggregateQuery(filter, aggregate, itemType, true);
     }
 
-    private Map<String, Long> aggregateQuery(final Condition filter, final BaseAggregate aggregate, final String itemType, final boolean
-            optimized) {
+    private Map<String, Long> aggregateQuery(final Condition filter, final BaseAggregate aggregate, final String itemType,
+            final boolean optimizedQuery) {
         return new InClassLoaderExecute<Map<String, Long>>(metricsService, this.getClass().getName() + ".aggregateQuery") {
 
             @Override
@@ -1572,7 +1572,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
 
                 // If the request is optimized then we don't need a global aggregation which is very slow and we can put the query with a
                 // filter on range items in the query block so we don't retrieve all the document before filtering the whole
-                if (optimized) {
+                if (optimizedQuery) {
                     for (AggregationBuilder aggregationBuilder : lastAggregation) {
                         builder.addAggregation(aggregationBuilder);
                     }
@@ -1600,7 +1600,7 @@ public class ElasticSearchPersistenceServiceImpl implements PersistenceService,
                 SearchResponse response = builder.execute().actionGet();
                 Aggregations aggregations = response.getAggregations();
                 if (aggregations != null) {
-                    if (optimized) {
+                    if (optimizedQuery) {
                         if (response.getHits() != null) {
                             results.put("_filtered", response.getHits().getTotalHits());
                         }

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/f579e142/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/PersistenceService.java
----------------------------------------------------------------------
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 c98eb17..ec22654 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
@@ -432,7 +432,7 @@ public interface PersistenceService {
      * @param itemType  the String representation of the item type we want to retrieve the count of, as defined by its class' {@code ITEM_TYPE} field
      * @return a Map associating aggregation dimension name as key and cardinality for that dimension as value
      */
-    Map<String, Long> aggregateQueryOptimized(Condition filter, BaseAggregate aggregate, String itemType);
+    Map<String, Long> aggregateWithOptimizedQuery(Condition filter, BaseAggregate aggregate, String itemType);
 
     /**
      * Updates the persistence's engine indices if needed.

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/f579e142/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java
----------------------------------------------------------------------
diff --git a/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java b/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java
index 3d68e05..25e2456 100644
--- a/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java
+++ b/rest/src/main/java/org/apache/unomi/rest/QueryServiceEndPoint.java
@@ -82,36 +82,25 @@ public class QueryServiceEndPoint {
      *
      * Retrieves the number of items with the specified type as defined by the Item subclass public field {@code ITEM_TYPE} and aggregated by possible values of the specified
      * property or, if the specified query is not {@code null}, perform that aggregate query.
-     * Also return the global count of document matching the {@code ITEM_TYPE}
+     * Also return the global count of document matching the {@code ITEM_TYPE} if you don't use {@code optimizedQuery} or set it to false,
+     * otherwise if {@code optimizedQuery} is set to true then it won't return the global count but the query will be executed much faster.
      *
      * @param type           the String representation of the item type we want to retrieve the count of, as defined by its class' {@code ITEM_TYPE} field
      * @param property       the property we're aggregating on, i.e. for each possible value of this property, we are counting how many items of the specified type have that value
-     * @param aggregateQuery the {@link AggregateQuery} specifying the aggregation that should be perfomed
+     * @param aggregateQuery the {@link AggregateQuery} specifying the aggregation that should be performed
+     * @param optimizedQuery the {@code optimizedQuery} specifying if we should optimized the aggregate query or not
      * @return a Map associating a specific value of the property to the cardinality of items with that value
      * @see Item Item for a discussion of {@code ITEM_TYPE}
      */
     @POST
     @Path("/{type}/{property}")
-    public Map<String, Long> getAggregate(@PathParam("type") String type, @PathParam("property") String property, AggregateQuery aggregateQuery) {
-        return queryService.getAggregate(type, property, aggregateQuery);
-    }
-
-    /**
-     * Retrieves the number of items with the specified type as defined by the Item subclass public field {@code ITEM_TYPE} and aggregated by possible values of the specified
-     * property or, if the specified query is not {@code null}, perform that aggregate query.
-     * This aggregate won't return the global count and should therefore be much faster than {@link #getAggregate(String, String, AggregateQuery)}
-     *
-     * @param type           the String representation of the item type we want to retrieve the count of, as defined by its class' {@code ITEM_TYPE} field
-     * @param property       the property we're aggregating on, i.e. for each possible value of this property, we are counting how many items of the specified type have that value
-     * @param aggregateQuery the {@link AggregateQuery} specifying the aggregation that should be perfomed
-     * @return a Map associating a specific value of the property to the cardinality of items with that value
-     * @see Item Item for a discussion of {@code ITEM_TYPE}
-     */
-    @POST
-    @Path("/{type}/{property}/optimized")
-    public Map<String, Long> getAggregateOptimized(@PathParam("type") String type, @PathParam("property") String property, AggregateQuery
-            aggregateQuery) {
-        return queryService.getAggregateOptimized(type, property, aggregateQuery);
+    public Map<String, Long> getAggregate(@PathParam("type") String type, @PathParam("property") String property,
+            @QueryParam("optimizedQuery") boolean optimizedQuery, AggregateQuery aggregateQuery) {
+        if (optimizedQuery) {
+            return queryService.getAggregateWithOptimizedQuery(type, property, aggregateQuery);
+        } else {
+            return queryService.getAggregate(type, property, aggregateQuery);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-unomi/blob/f579e142/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java
----------------------------------------------------------------------
diff --git a/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java b/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java
index 5da7076..bdbc9d1 100644
--- a/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java
+++ b/services/src/main/java/org/apache/unomi/services/services/QueryServiceImpl.java
@@ -62,7 +62,7 @@ public class QueryServiceImpl implements QueryService {
     }
 
     @Override
-    public Map<String, Long> getAggregateOptimized(String itemType, String property, AggregateQuery query) {
+    public Map<String, Long> getAggregateWithOptimizedQuery(String itemType, String property, AggregateQuery query) {
         return getAggregate(itemType, property, query, true);
     }
 
@@ -82,7 +82,7 @@ public class QueryServiceImpl implements QueryService {
         return persistenceService.queryCount(condition, itemType);
     }
 
-    private Map<String, Long> getAggregate(String itemType, String property, AggregateQuery query, boolean optimized) {
+    private Map<String, Long> getAggregate(String itemType, String property, AggregateQuery query, boolean optimizedQuery) {
         if (query != null) {
             // resolve condition
             if (query.getCondition() != null) {
@@ -115,8 +115,8 @@ public class QueryServiceImpl implements QueryService {
             }
 
             // fall back on terms aggregate
-            if (optimized) {
-                return persistenceService.aggregateQueryOptimized(query.getCondition(), baseAggregate, itemType);
+            if (optimizedQuery) {
+                return persistenceService.aggregateWithOptimizedQuery(query.getCondition(), baseAggregate, itemType);
             } else {
                 return persistenceService.aggregateQuery(query.getCondition(), baseAggregate, itemType);
             }