Browse Source

SQL: Proper handling of COUNT(field_name) and COUNT(DISTINCT field_name) (#37254)

* provide overriden `hashCode` and toString methods to account for `DISTINCT`
* change the analyzer for scenarios where `COUNT <field_name>` and `COUNT DISTINCT` have different paths
* defined a new `filter` aggregation encapsulating an `exists` query to filter out null or missing values
Andrei Stefan 6 years ago
parent
commit
4a92de214a

+ 32 - 1
docs/reference/sql/functions/aggs.asciidoc

@@ -53,11 +53,42 @@ COUNT(expression<1>)
 
 Returns the total number (count) of input values.
 
+In case of `COUNT(*)` or `COUNT(<literal>)`, _all_ values are considered (including `null` or missing ones).
+
+In case of `COUNT(<field_name>)` `null` values are not considered.
+
+
 ["source","sql",subs="attributes,macros"]
 --------------------------------------------------
 include-tagged::{sql-specs}/docs.csv-spec[aggCountStar]
 --------------------------------------------------
 
+
+[[sql-functions-aggs-count-all]]
+===== `COUNT(ALL)`
+
+.Synopsis:
+[source, sql]
+--------------------------------------------------
+COUNT(ALL field_name<1>)
+--------------------------------------------------
+
+*Input*:
+
+<1> a field name
+
+*Output*: numeric value
+
+.Description:
+
+Returns the total number (count) of all _non-null_ input values. `COUNT(<field_name>)` and `COUNT(ALL <field_name>)` are equivalent.
+
+["source","sql",subs="attributes,macros"]
+--------------------------------------------------
+include-tagged::{sql-specs}/docs.csv-spec[aggCountAll]
+--------------------------------------------------
+
+
 [[sql-functions-aggs-count-distinct]]
 ===== `COUNT(DISTINCT)`
 
@@ -75,7 +106,7 @@ COUNT(DISTINCT field_name<1>)
 
 .Description:
 
-Returns the total number of _distinct_ values in input values.
+Returns the total number of _distinct non-null_ values in input values.
 
 ["source","sql",subs="attributes,macros"]
 --------------------------------------------------

+ 56 - 1
x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec

@@ -309,4 +309,59 @@ SELECT HISTOGRAM(emp_no % 100, 10) AS h, COUNT(*) as c FROM test_emp GROUP BY h
 20             |10             
 10             |10             
 0              |10   
-;
+;
+
+countAll
+schema::all_names:l|c:l
+SELECT COUNT(ALL first_name) all_names, COUNT(*) c FROM test_emp;
+
+   all_names   |       c
+---------------+---------------
+90             |100
+;
+
+countAllCountTypesWithHaving
+schema::ln:l|dln:l|fn:l|dfn:l|ccc:l
+SELECT COUNT(last_name) ln, COUNT(distinct last_name) dln, COUNT(first_name) fn, COUNT(distinct first_name) dfn, COUNT(*) ccc FROM test_emp GROUP BY gender HAVING dln>5 AND ln>32 AND dfn>1 AND fn>1 AND ccc>5;
+
+      ln       |     dln     |      fn       |     dfn    |      ccc
+---------------+-------------+---------------+------------+-------------
+33             |32           |32             |32          |33
+57             |54           |48             |48          |57
+;
+
+aggCountEqualityFalse
+schema::areEqual:b|ln:l|dln:l
+SELECT COUNT(last_name)=COUNT(DISTINCT last_name) AS areEqual, COUNT(last_name) ln, COUNT(DISTINCT last_name) dln FROM test_emp;
+
+   areEqual    |      ln       |      dln
+---------------+---------------+---------------
+false          |100            |96
+;
+
+aggCountEqualityTrue
+schema::areEqual:b|fn:l|dfn:l
+SELECT COUNT(first_name)=COUNT(DISTINCT first_name) AS areEqual, COUNT(first_name) fn, COUNT(DISTINCT first_name) dfn FROM test_emp;
+
+   areEqual    |      fn       |      dfn
+---------------+---------------+---------------
+true           |90             |90
+;
+
+aggCountAllEquality
+schema::areEqual:b|afn:l
+SELECT COUNT(first_name)=COUNT(ALL first_name) AS areEqual, COUNT(ALL first_name) afn FROM test_emp;
+
+   areEqual    |      afn      
+---------------+---------------
+true           |90             
+;
+
+aggCountAllDifferentFields
+schema::areEqual:b|afn:l|aln:l
+SELECT COUNT(ALL last_name)=COUNT(ALL first_name) AS areEqual, COUNT(ALL first_name) afn, COUNT(ALL last_name) aln FROM test_emp;
+
+   areEqual    |      afn      |      aln
+---------------+---------------+---------------
+false          |90             |100
+;

+ 12 - 0
x-pack/plugin/sql/qa/src/main/resources/agg.sql-spec

@@ -210,6 +210,18 @@ SELECT gender g, languages l, COUNT(gender) c FROM "test_emp" GROUP BY g, l HAVI
 aggCountOnDistinctColumnAndHavingOnAliasAndFunctionConstantMultiGroupBy
 SELECT gender g, languages l, COUNT(DISTINCT last_name) c FROM "test_emp" GROUP BY g, l HAVING c > 5 AND COUNT(1) < 70 ORDER BY gender, languages;
 
+aggCount
+SELECT COUNT(last_name) c FROM test_emp;
+aggCountAndCountDistinct
+SELECT COUNT(last_name) c, COUNT(DISTINCT last_name) distinct_names FROM test_emp;
+aggCountAndCountDistinctWithHaving
+SELECT COUNT(last_name) c, COUNT(DISTINCT last_name) distinct_names, gender FROM test_emp GROUP BY gender HAVING distinct_names > 10 ORDER BY gender;
+aggCountMultiComparisonWithHaving
+SELECT COUNT(last_name) ln, COUNT(distinct last_name) dln, COUNT(first_name) fn, COUNT(distinct first_name) dfn, COUNT(*) ccc FROM test_emp GROUP BY gender HAVING dln>5 AND ln>32 AND dfn>1 AND fn>1 AND ccc>5 ORDER BY gender DESC;
+aggCountMultiComparisonWithHavingAndNullGrouping
+SELECT gender, COUNT(last_name) ln, COUNT(distinct last_name) dln, COUNT(first_name) fn, COUNT(distinct first_name) dfn, COUNT(*) ccc FROM test_emp GROUP BY gender HAVING dln>1 AND ln>1 AND dfn>1 AND fn>1 AND ccc>1 ORDER BY gender DESC;
+aggCountWithHavingAndWhere
+SELECT COUNT(last_name) c, COUNT(DISTINCT last_name) distinct_names, gender FROM test_emp WHERE salary > 65000 GROUP BY gender HAVING distinct_names > 10 ORDER BY gender;
 
 // MIN
 aggMinImplicit

+ 14 - 4
x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec

@@ -977,14 +977,24 @@ SELECT COUNT(*) AS count FROM emp;
 // end::aggCountStar
 ;
 
+aggCountAll
+// tag::aggCountAll
+SELECT COUNT(ALL last_name) AS count_all, COUNT(DISTINCT last_name) count_distinct FROM emp;
+
+   count_all   |  count_distinct  
+---------------+------------------
+100            |96   
+// end::aggCountAll
+;
+
 aggCountDistinct
 // tag::aggCountDistinct
 
-SELECT COUNT(DISTINCT hire_date) AS hires FROM emp;
+SELECT COUNT(DISTINCT hire_date) unique_hires, COUNT(hire_date) AS hires FROM emp;
 
-     hires      
----------------
-99       
+  unique_hires  |     hires
+----------------+---------------
+99              |100
 // end::aggCountDistinct
 ;
 

+ 19 - 2
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java

@@ -28,6 +28,7 @@ import org.elasticsearch.xpack.sql.expression.function.FunctionDefinition;
 import org.elasticsearch.xpack.sql.expression.function.FunctionRegistry;
 import org.elasticsearch.xpack.sql.expression.function.Functions;
 import org.elasticsearch.xpack.sql.expression.function.UnresolvedFunction;
+import org.elasticsearch.xpack.sql.expression.function.aggregate.Count;
 import org.elasticsearch.xpack.sql.expression.function.scalar.Cast;
 import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.ArithmeticOperation;
 import org.elasticsearch.xpack.sql.plan.TableIdentifier;
@@ -770,7 +771,15 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
                 List<Function> list = getList(seen, fName);
                 for (Function seenFunction : list) {
                     if (seenFunction != f && f.arguments().equals(seenFunction.arguments())) {
-                        return seenFunction;
+                        // Special check for COUNT: an already seen COUNT function will be returned only if its DISTINCT property
+                        // matches the one from the unresolved function to be checked. 
+                        if (seenFunction instanceof Count) {
+                            if (seenFunction.equals(f)){
+                                return seenFunction;
+                            }
+                        } else {
+                            return seenFunction;
+                        }
                     }
                 }
                 list.add(f);
@@ -808,7 +817,15 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
                     if (!list.isEmpty()) {
                         for (Function seenFunction : list) {
                             if (uf.arguments().equals(seenFunction.arguments())) {
-                                return seenFunction;
+                                // Special check for COUNT: an already seen COUNT function will be returned only if its DISTINCT property
+                                // matches the one from the unresolved function to be checked. 
+                                if (seenFunction instanceof Count) {
+                                    if (uf.sameAs((Count) seenFunction)) {
+                                        return seenFunction;
+                                    }
+                                } else {
+                                    return seenFunction;
+                                }
                             }
                         }
                     }

+ 4 - 0
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/extractor/MetricAggExtractor.java

@@ -9,6 +9,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket;
+import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter;
 import org.elasticsearch.search.aggregations.matrix.stats.MatrixStats;
 import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation;
 import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation.SingleValue;
@@ -83,6 +84,9 @@ public class MetricAggExtractor implements BucketExtractor {
             //    throw new SqlIllegalArgumentException("Invalid innerKey {} specified for aggregation {}", innerKey, name);
             //}
             return ((InternalNumericMetricsAggregation.MultiValue) agg).value(property);
+        } else if (agg instanceof InternalFilter) {
+            // COUNT(expr) and COUNT(ALL expr) uses this type of aggregation to account for non-null values only
+            return ((InternalFilter) agg).getDocCount();
         }
 
         Object v = agg.getProperty(property);

+ 10 - 1
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java

@@ -11,10 +11,11 @@ import org.elasticsearch.xpack.sql.expression.Attribute;
 import org.elasticsearch.xpack.sql.expression.Expression;
 import org.elasticsearch.xpack.sql.expression.Literal;
 import org.elasticsearch.xpack.sql.expression.Nullability;
+import org.elasticsearch.xpack.sql.expression.function.aggregate.Count;
 import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate;
 import org.elasticsearch.xpack.sql.session.Configuration;
-import org.elasticsearch.xpack.sql.tree.Source;
 import org.elasticsearch.xpack.sql.tree.NodeInfo;
+import org.elasticsearch.xpack.sql.tree.Source;
 import org.elasticsearch.xpack.sql.type.DataType;
 import org.elasticsearch.xpack.sql.util.StringUtils;
 
@@ -129,6 +130,14 @@ public class UnresolvedFunction extends Function implements Unresolvable {
     public boolean analyzed() {
         return analyzed;
     }
+    
+    public boolean sameAs(Count count) {
+        if (this.resolutionType == ResolutionType.DISTINCT && count.distinct()
+                || this.resolutionType == ResolutionType.STANDARD && count.distinct() == false) {
+            return true;
+        }
+        return false;
+    }
 
     @Override
     public DataType dataType() {

+ 38 - 5
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Count.java

@@ -5,14 +5,16 @@
  */
 package org.elasticsearch.xpack.sql.expression.function.aggregate;
 
-import java.util.List;
-
 import org.elasticsearch.xpack.sql.expression.Expression;
+import org.elasticsearch.xpack.sql.expression.Literal;
 import org.elasticsearch.xpack.sql.expression.NamedExpression;
-import org.elasticsearch.xpack.sql.tree.Source;
 import org.elasticsearch.xpack.sql.tree.NodeInfo;
+import org.elasticsearch.xpack.sql.tree.Source;
 import org.elasticsearch.xpack.sql.type.DataType;
 
+import java.util.List;
+import java.util.Objects;
+
 /**
  * Count the number of documents matched ({@code COUNT})
  * <strong>OR</strong> count the number of distinct values
@@ -53,17 +55,48 @@ public class Count extends AggregateFunction {
     public String functionId() {
         String functionId = id().toString();
         // if count works against a given expression, use its id (to identify the group)
-        if (field() instanceof NamedExpression) {
+        // in case of COUNT DISTINCT don't use the expression id to avoid possible duplicate IDs when COUNT and COUNT DISTINCT is used
+        // in the same query
+        if (!distinct() && field() instanceof NamedExpression) {
             functionId = ((NamedExpression) field()).id().toString();
         }
         return functionId;
     }
 
+    @Override
+    public String name() {
+        if (distinct()) {
+            StringBuilder sb = new StringBuilder(super.name());
+            sb.insert(sb.indexOf("(") + 1, "DISTINCT ");
+            return sb.toString();
+        }
+        return super.name();
+    }
+
     @Override
     public AggregateFunctionAttribute toAttribute() {
-        if (!distinct()) {
+        // COUNT(*) gets its value from the parent aggregation on which _count is called
+        if (field() instanceof Literal) {
             return new AggregateFunctionAttribute(source(), name(), dataType(), id(), functionId(), "_count");
         }
+        // COUNT(column) gets its value from a sibling aggregation (an exists filter agg) by calling its id and then _count on it
+        if (!distinct()) {
+            return new AggregateFunctionAttribute(source(), name(), dataType(), id(), functionId(), functionId() + "._count");
+        }
         return super.toAttribute();
     }
+    
+    @Override
+    public boolean equals(Object obj) {
+        if (false == super.equals(obj)) {
+            return false;
+        }
+        Count other = (Count) obj;
+        return Objects.equals(other.distinct(), distinct());
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(super.hashCode(), distinct());
+    }
 }

+ 1 - 1
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/grouping/GroupingFunction.java

@@ -51,7 +51,7 @@ public abstract class GroupingFunction extends Function {
     @Override
     public GroupingFunctionAttribute toAttribute() {
         if (lazyAttribute == null) {
-            // this is highly correlated with QueryFolder$FoldAggregate#addFunction (regarding the function name within the querydsl)
+            // this is highly correlated with QueryFolder$FoldAggregate#addAggFunction (regarding the function name within the querydsl)
             lazyAttribute = new GroupingFunctionAttribute(source(), name(), dataType(), id(), functionId());
         }
         return lazyAttribute;

+ 10 - 1
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryFolder.java

@@ -13,6 +13,7 @@ import org.elasticsearch.xpack.sql.expression.Attribute;
 import org.elasticsearch.xpack.sql.expression.Expression;
 import org.elasticsearch.xpack.sql.expression.Expressions;
 import org.elasticsearch.xpack.sql.expression.Foldables;
+import org.elasticsearch.xpack.sql.expression.Literal;
 import org.elasticsearch.xpack.sql.expression.NamedExpression;
 import org.elasticsearch.xpack.sql.expression.Order;
 import org.elasticsearch.xpack.sql.expression.function.Function;
@@ -380,7 +381,8 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
             // handle count as a special case agg
             if (f instanceof Count) {
                 Count c = (Count) f;
-                if (!c.distinct()) {
+                // COUNT(*) or COUNT(<literal>)
+                if (c.field() instanceof Literal) {
                     AggRef ref = groupingAgg == null ?
                             GlobalCountRef.INSTANCE :
                             new GroupByRef(groupingAgg.id(), Property.COUNT, null);
@@ -388,7 +390,14 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
                     Map<String, GroupByKey> pseudoFunctions = new LinkedHashMap<>(queryC.pseudoFunctions());
                     pseudoFunctions.put(functionId, groupingAgg);
                     return new Tuple<>(queryC.withPseudoFunctions(pseudoFunctions), new AggPathInput(f, ref));
+                // COUNT(<field_name>)
+                } else if (!c.distinct()) {
+                    LeafAgg leafAgg = toAgg(functionId, f);
+                    AggPathInput a = new AggPathInput(f, new MetricAggRef(leafAgg.id(), "doc_count", "_count"));
+                    queryC = queryC.with(queryC.aggs().addAgg(leafAgg));
+                    return new Tuple<>(queryC, a);
                 }
+                // the only variant left - COUNT(DISTINCT) - will be covered by the else branch below
             }
 
             AggPathInput aggInput = null;

+ 8 - 6
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryTranslator.java

@@ -62,6 +62,7 @@ import org.elasticsearch.xpack.sql.querydsl.agg.AndAggFilter;
 import org.elasticsearch.xpack.sql.querydsl.agg.AvgAgg;
 import org.elasticsearch.xpack.sql.querydsl.agg.CardinalityAgg;
 import org.elasticsearch.xpack.sql.querydsl.agg.ExtendedStatsAgg;
+import org.elasticsearch.xpack.sql.querydsl.agg.FilterExistsAgg;
 import org.elasticsearch.xpack.sql.querydsl.agg.GroupByDateHistogram;
 import org.elasticsearch.xpack.sql.querydsl.agg.GroupByKey;
 import org.elasticsearch.xpack.sql.querydsl.agg.GroupByNumericHistogram;
@@ -135,7 +136,7 @@ final class QueryTranslator {
             new MatrixStatsAggs(),
             new PercentilesAggs(),
             new PercentileRanksAggs(),
-            new DistinctCounts(),
+            new CountAggs(),
             new DateTimes()
             );
 
@@ -778,15 +779,16 @@ final class QueryTranslator {
     //
     // Agg translators
     //
-
-    static class DistinctCounts extends SingleValueAggTranslator<Count> {
+    
+    static class CountAggs extends SingleValueAggTranslator<Count> {
 
         @Override
         protected LeafAgg toAgg(String id, Count c) {
-            if (!c.distinct()) {
-                return null;
+            if (c.distinct()) {
+                return new CardinalityAgg(id, field(c));
+            } else {
+                return new FilterExistsAgg(id, field(c));
             }
-            return new CardinalityAgg(id, field(c));
         }
     }
 

+ 2 - 1
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/CardinalityAgg.java

@@ -15,7 +15,8 @@ public class CardinalityAgg extends LeafAgg {
         super(id, fieldName);
     }
 
-    @Override AggregationBuilder toBuilder() {
+    @Override
+    AggregationBuilder toBuilder() {
         return cardinality(id()).field(fieldName());
     }
 }

+ 26 - 0
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/FilterExistsAgg.java

@@ -0,0 +1,26 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.sql.querydsl.agg;
+
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.search.aggregations.AggregationBuilder;
+
+import static org.elasticsearch.search.aggregations.AggregationBuilders.filter;
+
+/**
+ * Aggregation builder for a "filter" aggregation encapsulating an "exists" query.
+ */
+public class FilterExistsAgg extends LeafAgg {
+
+    public FilterExistsAgg(String id, String fieldName) {
+        super(id, fieldName);
+    }
+
+    @Override
+    AggregationBuilder toBuilder() {
+        return filter(id(), QueryBuilders.existsQuery(fieldName()));
+    }
+}

+ 81 - 10
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java

@@ -5,7 +5,10 @@
  */
 package org.elasticsearch.xpack.sql.planner;
 
+import org.elasticsearch.index.query.ExistsQueryBuilder;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
+import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregationBuilder;
+import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder;
 import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
@@ -478,22 +481,90 @@ public class QueryTranslatorTests extends ESTestCase {
         assertEquals(DataType.DATE, field.dataType());
     }
     
-    public void testCountDistinctCardinalityFolder() {
-        PhysicalPlan p = optimizeAndPlan("SELECT COUNT(DISTINCT keyword) cnt FROM test GROUP BY bool HAVING cnt = 0");
+    public void testCountAndCountDistinctFolding() {
+        PhysicalPlan p = optimizeAndPlan("SELECT COUNT(DISTINCT keyword) dkey, COUNT(keyword) key FROM test");
         assertEquals(EsQueryExec.class, p.getClass());
         EsQueryExec ee = (EsQueryExec) p;
-        assertEquals(1, ee.output().size());
-        assertThat(ee.output().get(0).toString(), startsWith("cnt{a->"));
+        assertEquals(2, ee.output().size());
+        assertThat(ee.output().get(0).toString(), startsWith("dkey{a->"));
+        assertThat(ee.output().get(1).toString(), startsWith("key{a->"));
         
         Collection<AggregationBuilder> subAggs = ee.queryContainer().aggs().asAggBuilder().getSubAggregations();
-        assertEquals(1, subAggs.size());
+        assertEquals(2, subAggs.size());
         assertTrue(subAggs.toArray()[0] instanceof CardinalityAggregationBuilder);
+        assertTrue(subAggs.toArray()[1] instanceof FilterAggregationBuilder);
+
+        CardinalityAggregationBuilder cardinalityKeyword = (CardinalityAggregationBuilder) subAggs.toArray()[0];
+        assertEquals("keyword", cardinalityKeyword.field());
+        
+        FilterAggregationBuilder existsKeyword = (FilterAggregationBuilder) subAggs.toArray()[1];
+        assertTrue(existsKeyword.getFilter() instanceof ExistsQueryBuilder);
+        assertEquals("keyword", ((ExistsQueryBuilder) existsKeyword.getFilter()).fieldName());
+        
+        assertThat(ee.queryContainer().aggs().asAggBuilder().toString().replaceAll("\\s+", ""),
+                endsWith("{\"filter\":{\"exists\":{\"field\":\"keyword\",\"boost\":1.0}}}}}}"));
+    }
+    
+    public void testAllCountVariantsWithHavingGenerateCorrectAggregations() {
+        PhysicalPlan p = optimizeAndPlan("SELECT AVG(int), COUNT(keyword) ln, COUNT(distinct keyword) dln, COUNT(some.dotted.field) fn,"
+                + "COUNT(distinct some.dotted.field) dfn, COUNT(*) ccc FROM test GROUP BY bool "
+                + "HAVING dln > 3 AND ln > 32 AND dfn > 1 AND fn > 2 AND ccc > 5 AND AVG(int) > 50000");
+        assertEquals(EsQueryExec.class, p.getClass());
+        EsQueryExec ee = (EsQueryExec) p;
+        assertEquals(6, ee.output().size());
+        assertThat(ee.output().get(0).toString(), startsWith("AVG(int){a->"));
+        assertThat(ee.output().get(1).toString(), startsWith("ln{a->"));
+        assertThat(ee.output().get(2).toString(), startsWith("dln{a->"));
+        assertThat(ee.output().get(3).toString(), startsWith("fn{a->"));
+        assertThat(ee.output().get(4).toString(), startsWith("dfn{a->"));
+        assertThat(ee.output().get(5).toString(), startsWith("ccc{a->"));
         
-        CardinalityAggregationBuilder cardinalityAgg = (CardinalityAggregationBuilder) subAggs.toArray()[0];
-        assertEquals("keyword", cardinalityAgg.field());
+        Collection<AggregationBuilder> subAggs = ee.queryContainer().aggs().asAggBuilder().getSubAggregations();
+        assertEquals(5, subAggs.size());
+        assertTrue(subAggs.toArray()[0] instanceof AvgAggregationBuilder);
+        assertTrue(subAggs.toArray()[1] instanceof FilterAggregationBuilder);
+        assertTrue(subAggs.toArray()[2] instanceof CardinalityAggregationBuilder);
+        assertTrue(subAggs.toArray()[3] instanceof FilterAggregationBuilder);
+        assertTrue(subAggs.toArray()[4] instanceof CardinalityAggregationBuilder);
+        
+        AvgAggregationBuilder avgInt = (AvgAggregationBuilder) subAggs.toArray()[0];
+        assertEquals("int", avgInt.field());
+        
+        FilterAggregationBuilder existsKeyword = (FilterAggregationBuilder) subAggs.toArray()[1];
+        assertTrue(existsKeyword.getFilter() instanceof ExistsQueryBuilder);
+        assertEquals("keyword", ((ExistsQueryBuilder) existsKeyword.getFilter()).fieldName());
+        
+        CardinalityAggregationBuilder cardinalityKeyword = (CardinalityAggregationBuilder) subAggs.toArray()[2];
+        assertEquals("keyword", cardinalityKeyword.field());
+        
+        FilterAggregationBuilder existsDottedField = (FilterAggregationBuilder) subAggs.toArray()[3];
+        assertTrue(existsDottedField.getFilter() instanceof ExistsQueryBuilder);
+        assertEquals("some.dotted.field", ((ExistsQueryBuilder) existsDottedField.getFilter()).fieldName());
+        
+        CardinalityAggregationBuilder cardinalityDottedField = (CardinalityAggregationBuilder) subAggs.toArray()[4];
+        assertEquals("some.dotted.field", cardinalityDottedField.field());
+
         assertThat(ee.queryContainer().aggs().asAggBuilder().toString().replaceAll("\\s+", ""),
-                endsWith("{\"buckets_path\":{\"a0\":\"" + cardinalityAgg.getName() +"\"},\"script\":{"
-                        + "\"source\":\"InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.eq(params.a0,params.v0))\","
-                        + "\"lang\":\"painless\",\"params\":{\"v0\":0}},\"gap_policy\":\"skip\"}}}}}"));
+                endsWith("{\"buckets_path\":{"
+                        + "\"a0\":\"" + cardinalityKeyword.getName() + "\","
+                        + "\"a1\":\"" + existsKeyword.getName() + "._count\","
+                        + "\"a2\":\"" + cardinalityDottedField.getName() + "\","
+                        + "\"a3\":\"" + existsDottedField.getName() + "._count\","
+                        + "\"a4\":\"_count\","
+                        + "\"a5\":\"" + avgInt.getName() + "\"},"
+                        + "\"script\":{\"source\":\""
+                        + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and("
+                        +   "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and("
+                        +     "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and("
+                        +       "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and("
+                        +         "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and("
+                        +           "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a0,params.v0)),"
+                        +           "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a1,params.v1)))),"
+                        +         "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a2,params.v2)))),"
+                        +       "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a3,params.v3)))),"
+                        +     "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a4,params.v4)))),"
+                        +   "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a5,params.v5))))\","
+                        + "\"lang\":\"painless\",\"params\":{\"v0\":3,\"v1\":32,\"v2\":1,\"v3\":2,\"v4\":5,\"v5\":50000}},"
+                        + "\"gap_policy\":\"skip\"}}}}}"));
     }
 }