korlov42 commented on a change in pull request #9276:
URL: https://github.com/apache/ignite/pull/9276#discussion_r687531145



##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdRowCount.java
##########
@@ -117,6 +117,13 @@ public double getRowCount(IgniteSortedIndexSpool rel, 
RelMetadataQuery mq) {
         return rel.estimateRowCount(mq);
     }
 
+    /**
+     * Estimation of row count for table scan.
+     */
+//    public double getRowCount(IgniteTableScan rel, RelMetadataQuery mq) {

Review comment:
       commented method

##########
File path: 
modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java
##########
@@ -829,6 +850,22 @@ public String name() {
         }
     }
 
+    public static class TestIndex extends IgniteIndex {

Review comment:
       Why do we need such an extension?

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteStatisticsImpl.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.schema;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelReferentialConstraint;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.util.ImmutableBitSet;
+import 
org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
+import org.apache.ignite.internal.processors.query.stat.ColumnStatistics;
+import org.apache.ignite.internal.processors.query.stat.ObjectStatisticsImpl;
+
+/** Calcite statistics wrapper. */
+public class IgniteStatisticsImpl implements Statistic {
+    /** Internal statistics implementation. */
+    private final ObjectStatisticsImpl statistics;
+
+    /**
+     * Constructor.
+     *
+     * @param statistics Internal object statistics or {@code null}.
+     */
+    public IgniteStatisticsImpl(ObjectStatisticsImpl statistics) {
+        this.statistics = statistics;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Double getRowCount() {
+        long rows = (statistics == null) ? 1000 : statistics.rowCount();
+
+        return (double)rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isKey(ImmutableBitSet cols) {
+        return false;

Review comment:
       let's throw an UnsupportedOperationException here and below

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdColumnOrigins.java
##########
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.metadata;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.Exchange;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.metadata.RelColumnOrigin;
+import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.util.BuiltInMethod;
+import 
org.apache.ignite.internal.processors.query.calcite.rel.ProjectableFilterableTableScan;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * RelMdColumnOrigins supplies a default implementation of
+ * {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra.
+ */
+public class IgniteMdColumnOrigins implements 
MetadataHandler<BuiltInMetadata.ColumnOrigin> {
+    /** */
+    public static final RelMetadataProvider SOURCE = 
ReflectiveRelMetadataProvider.reflectiveSource(
+            BuiltInMethod.COLUMN_ORIGIN.method, new IgniteMdColumnOrigins());
+
+    /** {@inheritDoc} */
+    @Override public MetadataDef<BuiltInMetadata.ColumnOrigin> getDef() {
+        return BuiltInMetadata.ColumnOrigin.DEF;
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(Aggregate rel, 
RelMetadataQuery mq, int iOutputColumn) {
+        if (iOutputColumn < rel.getGroupCount()) {
+            // get actual index of Group columns.
+            return mq.getColumnOrigins(rel.getInput(), 
rel.getGroupSet().asList().get(iOutputColumn));
+        }
+
+        // Aggregate columns are derived from input columns
+        AggregateCall call = rel.getAggCallList().get(iOutputColumn - 
rel.getGroupCount());
+
+        final Set<RelColumnOrigin> set = new HashSet<>();
+
+        for (Integer iInput : call.getArgList()) {
+            Set<RelColumnOrigin> inputSet = 
mq.getColumnOrigins(rel.getInput(), iInput);
+            inputSet = createDerivedColumnOrigins(inputSet);
+
+            if (inputSet != null)
+                set.addAll(inputSet);
+
+        }
+        return set;
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(Join rel, 
RelMetadataQuery mq,
+        int iOutputColumn) {
+        int nLeftColumns = rel.getLeft().getRowType().getFieldList().size();
+        Set<RelColumnOrigin> set;
+        boolean derived = false;
+
+        if (iOutputColumn < nLeftColumns) {
+            set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn);
+
+            if (rel.getJoinType().generatesNullsOnLeft())
+                derived = true;
+
+        }
+        else {
+            set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - 
nLeftColumns);
+
+            if (rel.getJoinType().generatesNullsOnRight())
+                derived = true;
+        }
+
+        if (derived) {
+            // nulls are generated due to outer join; that counts
+            // as derivation
+            set = createDerivedColumnOrigins(set);
+        }
+        return set;
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(SetOp rel, 
RelMetadataQuery mq, int iOutputColumn) {
+        final Set<RelColumnOrigin> set = new HashSet<>();
+
+        for (RelNode input : rel.getInputs()) {
+            Set<RelColumnOrigin> inputSet = mq.getColumnOrigins(input, 
iOutputColumn);
+
+            if (inputSet == null)
+                return null;
+
+            set.addAll(inputSet);
+        }
+
+        return set;
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(Project rel,
+        final RelMetadataQuery mq, int iOutputColumn) {
+        final RelNode input = rel.getInput();
+        RexNode rexNode = rel.getProjects().get(iOutputColumn);
+
+        if (rexNode instanceof RexInputRef) {
+            // Direct reference:  no derivation added.
+            RexInputRef inputRef = (RexInputRef) rexNode;
+
+            return mq.getColumnOrigins(input, inputRef.getIndex());
+        }
+        // Anything else is a derivation, possibly from multiple columns.
+        final Set<RelColumnOrigin> set = getMultipleColumns(rexNode, input, 
mq);
+
+        return createDerivedColumnOrigins(set);
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(Calc rel, final 
RelMetadataQuery mq, int iOutputColumn) {
+        final RelNode input = rel.getInput();
+        final RexShuttle rexShuttle = new RexShuttle() {
+            @Override public RexNode visitLocalRef(RexLocalRef localRef) {
+                return rel.getProgram().expandLocalRef(localRef);
+            }
+        };
+        final List<RexNode> projects = new ArrayList<>();
+
+        for (RexNode rex: rexShuttle.apply(rel.getProgram().getProjectList()))
+            projects.add(rex);
+
+        final RexNode rexNode = projects.get(iOutputColumn);
+
+        if (rexNode instanceof RexInputRef) {
+            // Direct reference:  no derivation added.
+            RexInputRef inputRef = (RexInputRef) rexNode;
+            return mq.getColumnOrigins(input, inputRef.getIndex());
+        }
+        // Anything else is a derivation, possibly from multiple columns.
+        final Set<RelColumnOrigin> set = getMultipleColumns(rexNode, input, 
mq);
+
+        return createDerivedColumnOrigins(set);
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(Filter rel, 
RelMetadataQuery mq, int iOutputColumn) {
+        return mq.getColumnOrigins(rel.getInput(), iOutputColumn);
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(Sort rel, 
RelMetadataQuery mq, int iOutputColumn) {
+        return mq.getColumnOrigins(rel.getInput(), iOutputColumn);
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(TableModify rel, 
RelMetadataQuery mq, int iOutputColumn) {
+        return mq.getColumnOrigins(rel.getInput(), iOutputColumn);
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(Exchange rel, 
RelMetadataQuery mq, int iOutputColumn) {
+        return mq.getColumnOrigins(rel.getInput(), iOutputColumn);
+    }
+
+    /** */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(
+        TableFunctionScan rel,
+        RelMetadataQuery mq,
+        int iOutputColumn
+    ) {
+        Set<RelColumnMapping> mappings = rel.getColumnMappings();
+
+        if (mappings == null) {
+            if (!rel.getInputs().isEmpty()) {
+                // This is a non-leaf transformation:  say we don't
+                // know about origins, because there are probably
+                // columns below.
+                return null;
+            }
+            else {
+                // This is a leaf transformation: say there are for sure no
+                // column origins.
+                return Collections.emptySet();
+            }
+        }
+
+        final Set<RelColumnOrigin> set = new HashSet<>();
+
+        for (RelColumnMapping mapping : mappings) {
+            if (mapping.iOutputColumn != iOutputColumn)
+                continue;
+
+            final RelNode input = rel.getInputs().get(mapping.iInputRel);
+            final int column = mapping.iInputColumn;
+            Set<RelColumnOrigin> origins = mq.getColumnOrigins(input, column);
+
+            if (origins == null)
+                return null;
+
+            if (mapping.derived)
+                origins = createDerivedColumnOrigins(origins);
+
+            set.addAll(origins);
+        }
+
+        return set;
+    }
+
+    /**
+     * Get column origins.
+     *
+     * @param rel Rel to get origins from.
+     * @param mq Rel metadata query.
+     * @param iOutputColumn Column idx.
+     * @return Set of column origins.
+     */
+    public @Nullable Set<RelColumnOrigin> getColumnOrigins(
+        ProjectableFilterableTableScan rel,

Review comment:
       Do you have any tests on this?

##########
File path: 
modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/AbstractPlannerTest.java
##########
@@ -653,11 +656,17 @@ protected static IgniteSchema createSchema(TestTable... 
tbls) {
         private final RewindabilityTrait rewindable;
 
         /** */
-        private final double rowCnt;
+        private IgniteStatisticsImpl statistics;
 
         /** */
         private final TableDescriptor desc;
 
+        /** */
+        private ColocationGroup colocationGroup;

Review comment:
       why do we need colocation group and distribution here?

##########
File path: 
modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java
##########
@@ -89,6 +91,72 @@
         return containsSubPlan("IgniteIndexScan(table=[[" + schema + ", " + 
tblName + "]], index=[" + idxName + ']');
     }
 
+    /**
+     * Ignite cost mather.
+     *
+     * @param cost TestCost to match.
+     * @return Matcher.
+     */
+    public static Matcher<String> 
containsCost(ServerStatisticsIntegrationTest.TestCost cost) {

Review comment:
       I don't think it's a good idea to have method like this. Otherwise we 
will be needed to fix all the tests uses this check when we start to tweak 
cost's weights.

##########
File path: 
modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java
##########
@@ -0,0 +1,695 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.integration;
+
+import java.math.BigInteger;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import 
org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.QueryChecker;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import 
org.apache.ignite.internal.processors.query.stat.IgniteStatisticsManager;
+import org.apache.ignite.internal.processors.query.stat.StatisticsKey;
+import org.apache.ignite.internal.processors.query.stat.StatisticsTarget;
+import 
org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+/**
+ * Tests for server side statistics usage.
+ */
+public class ServerStatisticsIntegrationTest extends 
AbstractBasicIntegrationTest {

Review comment:
       I don't think it's a good idea to check statistics usage with an 
integration tests.

##########
File path: 
modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java
##########
@@ -35,11 +35,13 @@
 import 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache;
 import 
org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.query.QueryEngine;
+import 
org.apache.ignite.internal.processors.query.calcite.integration.ServerStatisticsIntegrationTest;

Review comment:
       general queryChecker has dependency on particular test

##########
File path: 
modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java
##########
@@ -0,0 +1,695 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.integration;
+
+import java.math.BigInteger;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.query.QueryEngine;
+import 
org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor;
+import org.apache.ignite.internal.processors.query.calcite.QueryChecker;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import 
org.apache.ignite.internal.processors.query.stat.IgniteStatisticsManager;
+import org.apache.ignite.internal.processors.query.stat.StatisticsKey;
+import org.apache.ignite.internal.processors.query.stat.StatisticsTarget;
+import 
org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+/**
+ * Tests for server side statistics usage.
+ */
+public class ServerStatisticsIntegrationTest extends 
AbstractBasicIntegrationTest {
+    /** Server instance. */
+    private IgniteEx srv;
+
+    /** All types table row count. */
+    private static final int ROW_COUNT = 100;
+
+    /** All types table nullable fields. */
+    private static final String[] NULLABLE_FIELDS = {
+        "string_field",
+        "boolean_obj_field",
+        "short_obj_field",
+        "integer_field",
+        "long_obj_field",
+        "float_obj_field",
+        "double_obj_field",
+    };
+
+    /** All types table non nullable fields. */
+    private static final String[] NON_NULLABLE_FIELDS = {
+        "short_field",
+        "int_field",
+        "long_field",
+        "float_field",
+        "double_field"
+    };
+
+    /** All types table numeric fields. */
+    private static final String[] NUMERIC_FIELDS = {
+        "short_obj_field",
+        "integer_field",
+        "long_obj_field",
+        "float_obj_field",
+        "double_obj_field",
+        "short_field",
+        "int_field",
+        "long_field",
+        "float_field",
+        "double_field"
+    };
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        createAndPopulateAllTypesTable(0, ROW_COUNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int nodeCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() {
+        cleanQueryPlanCache();
+    }
+
+    /**
+     * Run select and check that cost take statisitcs in account:
+     * 1) without statistics;
+     * 2) with statistics;
+     * 3) after deleting statistics.
+     */
+    @Test
+    public void testQueryCostWithStatistics() throws IgniteCheckedException {
+        createAndPopulateTable();
+        StatisticsKey key = new StatisticsKey("PUBLIC", "PERSON");
+        srv = ignite(0);
+
+        TestCost costWoStats = new TestCost(1000., 1000., null, null, null);
+
+        assertQuerySrv("select count(name) from 
person").matches(QueryChecker.containsCost(costWoStats)).check();
+
+        clearQryCache(srv);
+
+        collectStatistics(srv, key);
+
+        TestCost costWithStats = new TestCost(5., 5., null, null, null);
+
+        assertQuerySrv("select count(name) from 
person").matches(QueryChecker.containsCost(costWithStats)).check();
+
+        statMgr(srv).dropStatistics(new StatisticsTarget(key));
+        clearQryCache(srv);
+
+        assertQuerySrv("select count(name) from 
person").matches(QueryChecker.containsCost(costWoStats)).check();
+    }
+
+    /**
+     * Check is null/is not null conditions for nullable and non nullable 
fields.
+     */
+    @Test
+    public void testNullConditions() throws IgniteCheckedException {
+        StatisticsKey key = new StatisticsKey("PUBLIC", "ALL_TYPES");
+        srv = ignite(0);
+
+        collectStatistics(srv, key);
+
+        String sql = "select * from all_types ";
+
+        for (String nullableField : NULLABLE_FIELDS) {
+            assertQuerySrv(sql + "where " + nullableField + " is null")
+                .matches(QueryChecker.containsRowCount(25.)).check();
+
+            assertQuerySrv(sql + "where " + nullableField + " is not null")
+                .matches(QueryChecker.containsRowCount(75.)).check();
+        }
+
+        for (String nonNullableField : NON_NULLABLE_FIELDS) {
+            assertQuerySrv(sql + "where " + nonNullableField + " is null")
+                .matches(QueryChecker.containsRowCount(1.)).check();
+
+            assertQuerySrv(sql + "where " + nonNullableField + " is not null")
+                .matches(QueryChecker.containsRowCount(ROW_COUNT)).check();
+        }
+    }
+
+    /**
+     * Test multiple condition for the same query.
+     *
+     * @throws IgniteCheckedException In case of errors.
+     */
+    @Test
+    public void testMultipleConditionQuery() throws IgniteCheckedException {
+        StatisticsKey key = new StatisticsKey("PUBLIC", "ALL_TYPES");
+        srv = ignite(0);
+
+        collectStatistics(srv, key);
+
+        Set<String> nonNullableFields = new 
HashSet(Arrays.asList(NON_NULLABLE_FIELDS));
+
+        for (String numericField : NUMERIC_FIELDS) {
+            double allRowCnt = (nonNullableFields.contains(numericField)) ? 
(double)ROW_COUNT : 0.75 * ROW_COUNT;
+
+            String fieldSql = String.format("select * from all_types where %s 
> -100 and %s > 0", numericField, numericField);
+
+            
assertQuerySrv(fieldSql).matches(QueryChecker.containsRowCount(allRowCnt)).check();
+
+            fieldSql = String.format("select * from all_types where %s < 1000 
and %s < 101", numericField, numericField);
+
+            
assertQuerySrv(fieldSql).matches(QueryChecker.containsRowCount(allRowCnt)).check();
+
+            fieldSql = String.format("select * from all_types where %s > -100 
and %s < 1000", numericField, numericField);
+
+            
assertQuerySrv(fieldSql).matches(QueryChecker.containsRowCount(allRowCnt)).check();
+        }
+    }
+
+    /**
+     * Check range condition with not null conditions.
+     *
+     * @throws IgniteCheckedException In case of error.
+     */
+    @Test
+    public void testNonNullMultipleConditionQuery() throws 
IgniteCheckedException {
+        StatisticsKey key = new StatisticsKey("PUBLIC", "ALL_TYPES");
+        srv = ignite(0);
+
+        collectStatistics(srv, key);
+
+        Set<String> nonNullableFields = new 
HashSet(Arrays.asList(NON_NULLABLE_FIELDS));
+
+        // time
+        String timeSql = "select * from all_types where time_field is not 
null";
+
+        
assertQuerySrv(timeSql).matches(QueryChecker.containsRowCount(ROW_COUNT * 
0.75)).check();
+
+        timeSql += " and time_field > '00:00:00'";
+
+        
assertQuerySrv(timeSql).matches(QueryChecker.containsRowCount(ROW_COUNT * 
0.75)).check();
+
+        // date
+        String dateSql = "select * from all_types where date_field is not 
null";
+
+        
assertQuerySrv(dateSql).matches(QueryChecker.containsRowCount(ROW_COUNT * 
0.75)).check();
+
+        dateSql += " and date_field > '1000-01-01'";
+
+        
assertQuerySrv(dateSql).matches(QueryChecker.containsRowCount(ROW_COUNT * 
0.75)).check();
+
+        // timestamp
+        String timestampSql = "select * from all_types where timestamp_field 
is not null ";
+
+        
assertQuerySrv(timestampSql).matches(QueryChecker.containsRowCount(ROW_COUNT * 
0.75)).check();
+
+        timestampSql += " and timestamp_field > '1000-01-10 11:59:59'";
+
+        
assertQuerySrv(timestampSql).matches(QueryChecker.containsRowCount(ROW_COUNT * 
0.75)).check();
+
+        // numeric fields
+        for (String numericField : NUMERIC_FIELDS) {
+            double allRowCnt = (nonNullableFields.contains(numericField)) ? 
(double)ROW_COUNT : 0.75 * ROW_COUNT;
+
+            String fieldSql = String.format("select * from all_types where %s 
is not null",
+                numericField, numericField);
+
+            
assertQuerySrv(fieldSql).matches(QueryChecker.containsRowCount(allRowCnt)).check();
+
+            fieldSql = String.format("select * from all_types where %s is not 
null and %s > 0",
+                numericField, numericField);
+
+            
assertQuerySrv(fieldSql).matches(QueryChecker.containsRowCount(allRowCnt)).check();
+        }
+    }
+
+    /**
+     * Check condition with projections:
+     *
+     * 1) Condition on the one of fields in select list.
+     * 2) Confition on the field not from select list.
+     *
+     * @throws IgniteCheckedException In case of errors.
+     */
+    @Test
+    public void testProjections() throws IgniteCheckedException {
+        StatisticsKey key = new StatisticsKey("PUBLIC", "ALL_TYPES");
+        srv = ignite(0);
+
+        collectStatistics(srv, key);
+
+        String sql = "select %s, %s from all_types where %s < " + ROW_COUNT;
+
+        String sql2 = "select %s from all_types where %s >= " + (-ROW_COUNT);
+
+        Set<String> nonNullableFields = new 
HashSet(Arrays.asList(NON_NULLABLE_FIELDS));
+
+        for (int firstFieldIdx = 0; firstFieldIdx < NUMERIC_FIELDS.length - 1; 
firstFieldIdx++) {
+            String firstField = NUMERIC_FIELDS[firstFieldIdx];
+            double firstAllRowCnt = (nonNullableFields.contains(firstField)) ? 
(double)ROW_COUNT : 0.75 * ROW_COUNT;
+
+            for (int secFieldIdx = firstFieldIdx + 1; secFieldIdx < 
NUMERIC_FIELDS.length; secFieldIdx++) {
+                String secField = NUMERIC_FIELDS[secFieldIdx];
+
+                double secAllRowCnt = (nonNullableFields.contains(secField)) ? 
(double)ROW_COUNT : 0.75 * ROW_COUNT;
+
+                String qry = String.format(sql, firstField, secField, 
secField);
+
+                
assertQuerySrv(qry).matches(QueryChecker.containsRowCount(secAllRowCnt)).check();
+
+                qry = String.format(sql, firstField, secField, firstField);
+
+                
assertQuerySrv(qry).matches(QueryChecker.containsRowCount(firstAllRowCnt)).check();
+
+                qry = String.format(sql2, firstField, secField);
+
+                
assertQuerySrv(qry).matches(QueryChecker.containsRowCount(secAllRowCnt)).check();
+
+                qry = String.format(sql2, secField, firstField);
+
+                
assertQuerySrv(qry).matches(QueryChecker.containsRowCount(firstAllRowCnt)).check();
+            }
+        }
+    }
+
+    /**
+     * Test not null counting with two range conjuncted condition on one and 
two columns.
+     *
+     * @throws IgniteCheckedException In case of errors.
+     */
+    @Test
+    public void testNotNullCountingSelectivity() throws IgniteCheckedException 
{
+        StatisticsKey key = new StatisticsKey("PUBLIC", "ALL_TYPES");
+        srv = ignite(0);
+
+        collectStatistics(srv, key);
+
+        Set<String> nonNullableFields = new 
HashSet(Arrays.asList(NON_NULLABLE_FIELDS));
+
+        for (String numericField : NUMERIC_FIELDS) {
+            double allRowCnt = (nonNullableFields.contains(numericField)) ? 
(double)ROW_COUNT : 0.75 * ROW_COUNT;
+
+            assertQuerySrv(String.format("select * from all_types where " +
+                "%s > %d and %s < %d", numericField, -1, numericField, 101))
+                .matches(QueryChecker.containsRowCount(allRowCnt)).check();
+
+            assertQuerySrv(String.format("select * from all_types where " +
+                "(%s > %d and %s < %d) or (string_field > 'string_field_value' 
and string_field < 'string_field_value999')",
+                numericField, -1, numericField, 101))
+                .matches(QueryChecker.containsRowCount(allRowCnt)).check();
+        }
+    }
+
+    /**
+     * Test disjunctions selectivity for each column:
+     * 1) with select all conditions
+     * 2) with select none conditions
+     * 3) with is null or select all conditions
+     * 4) with is null or select none conditions
+     *
+     * @throws IgniteCheckedException In case of errors.
+     */
+    @Test
+    public void testDisjunctionSelectivity() throws IgniteCheckedException {
+        StatisticsKey key = new StatisticsKey("PUBLIC", "ALL_TYPES");
+        srv = ignite(0);
+
+        collectStatistics(srv, key);
+
+        Set<String> nonNullableFields = new 
HashSet(Arrays.asList(NON_NULLABLE_FIELDS));
+        for (String numericField : NUMERIC_FIELDS) {
+            double allRowCnt = (nonNullableFields.contains(numericField)) ? 
(double)ROW_COUNT : 0.75 * ROW_COUNT;
+
+            assertQuerySrv(String.format("select * from all_types where " +
+                "%s > %d or %s < %d", numericField, -1, numericField, 101))
+                .matches(QueryChecker.containsRowCount(allRowCnt)).check();
+
+            assertQuerySrv(String.format("select * from all_types where " +
+                "%s > %d or %s < %d", numericField, 101, numericField, -1))
+                .matches(QueryChecker.containsRowCount(1.)).check();
+
+            assertQuerySrv(String.format("select * from all_types where " +
+                "%s > %d or %s is null", numericField, -1, numericField))
+                .matches(QueryChecker.containsRowCount(ROW_COUNT)).check();
+
+
+            assertQuerySrv(String.format("select * from all_types where " +
+                "%s > %d or %s is null", numericField, 101, numericField))
+                .matches(QueryChecker.containsRowCount(
+                    nonNullableFields.contains(numericField) ? 1. : 
(double)ROW_COUNT * 0.25)).check();
+        }
+    }
+
+    /**
+     * Check randge with min/max borders.
+     */
+    @Test
+    public void testBorders() throws IgniteCheckedException {
+        StatisticsKey key = new StatisticsKey("PUBLIC", "ALL_TYPES");
+        srv = ignite(0);
+
+        collectStatistics(srv, key);
+
+        // time
+        String timeSql = "select * from all_types where time_field > 
'00:00:00'";
+
+        
assertQuerySrv(timeSql).matches(QueryChecker.containsRowCount(ROW_COUNT * 
0.75)).check();
+
+        // date
+        String dateSql = "select * from all_types where date_field > 
'1000-01-10'";
+
+        
assertQuerySrv(dateSql).matches(QueryChecker.containsRowCount(ROW_COUNT * 
0.75)).check();
+
+        // timestamp
+        String timestampSql = "select * from all_types where timestamp_field > 
'1000-01-10 11:59:59'";
+
+        
assertQuerySrv(timestampSql).matches(QueryChecker.containsRowCount(ROW_COUNT * 
0.75)).check();
+
+        String sql = "select * from all_types ";
+
+        Set<String> nonNullableFields = new 
HashSet(Arrays.asList(NON_NULLABLE_FIELDS));
+        for (String numericField : NUMERIC_FIELDS) {
+            double allRowCnt = (nonNullableFields.contains(numericField)) ? 
(double)ROW_COUNT : 0.75 * ROW_COUNT;
+
+            String fieldSql = sql + "where " + numericField;
+
+            assertQuerySrv(fieldSql + " <  
-1").matches(QueryChecker.containsRowCount(1.)).check();
+            assertQuerySrv(fieldSql + " <  
0").matches(QueryChecker.containsRowCount(1.)).check();
+            assertQuerySrv(fieldSql + " <=  
0").matches(QueryChecker.containsRowCount(1.)).check();
+            assertQuerySrv(fieldSql + " >=  
0").matches(QueryChecker.containsRowCount(allRowCnt)).check();
+            assertQuerySrv(fieldSql + " > 
0").matches(QueryChecker.containsRowCount(allRowCnt)).check();
+
+            assertQuerySrv(fieldSql + " > 
101").matches(QueryChecker.containsRowCount(1.)).check();
+            assertQuerySrv(fieldSql + " > 
100").matches(QueryChecker.containsRowCount(1.)).check();
+            assertQuerySrv(fieldSql + " >= 
100").matches(QueryChecker.containsRowCount(1.)).check();
+            assertQuerySrv(fieldSql + " <= 
100").matches(QueryChecker.containsRowCount(allRowCnt)).check();
+            assertQuerySrv(fieldSql + " < 
100").matches(QueryChecker.containsRowCount(allRowCnt)).check();
+        }
+    }
+
+    /**
+     * Clear query cache in specified node.
+     *
+     * @param ign Ignite node to clear calcite query cache on.
+     */
+    protected void clearQryCache(IgniteEx ign) {
+        CalciteQueryProcessor qryProc = 
(CalciteQueryProcessor)Commons.lookupComponent(
+            (ign).context(), QueryEngine.class);
+
+        qryProc.queryPlanCache().clear();
+    }
+
+    /**
+     * Collect statistics by speicifed key on specified node.
+     *
+     * @param ign Node to collect statistics on.
+     * @param key Statistics key to collect statistics by.
+     * @throws IgniteCheckedException In case of errors.
+     */
+    protected void collectStatistics(IgniteEx ign, StatisticsKey key) throws 
IgniteCheckedException {
+        IgniteStatisticsManager statMgr = statMgr(ign);
+
+        statMgr.collectStatistics(new StatisticsObjectConfiguration(key));
+
+        assertTrue(GridTestUtils.waitForCondition(() -> 
statMgr.getLocalStatistics(key) != null, 1000));
+    }
+
+    /**
+     * Get statistics manager.
+     *
+     * @param ign Node to get statistics manager from.
+     * @return IgniteStatisticsManager.
+     */
+    protected IgniteStatisticsManager statMgr(IgniteEx ign) {
+        IgniteH2Indexing indexing = 
(IgniteH2Indexing)ign.context().query().getIndexing();
+
+        return indexing.statsManager();
+    }
+
+    /** */
+    protected QueryChecker assertQuerySrv(String qry) {
+        return new QueryChecker(qry) {
+            @Override protected QueryEngine getEngine() {
+                return Commons.lookupComponent(srv.context(), 
QueryEngine.class);
+            }
+        };
+    }
+
+    /**
+     * Create (if not exists) and populate cache with all types.
+     *
+     * @param start first key idx.
+     * @param count rows count.

Review comment:
       ```suggestion
        * @param start First key idx.
        * @param count Rows count.
   ```

##########
File path: 
modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/StatisticsPlannerTest.java
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.calcite.planner;
+
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.HashMap;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.util.ImmutableIntList;
+import 
org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteIndex;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteSchema;
+import 
org.apache.ignite.internal.processors.query.calcite.schema.IgniteStatisticsImpl;
+import 
org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import 
org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import 
org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
+import org.apache.ignite.internal.processors.query.stat.ColumnStatistics;
+import org.apache.ignite.internal.processors.query.stat.ObjectStatisticsImpl;
+import org.h2.value.ValueBoolean;
+import org.h2.value.ValueByte;
+import org.h2.value.ValueDate;
+import org.h2.value.ValueDouble;
+import org.h2.value.ValueFloat;
+import org.h2.value.ValueInt;
+import org.h2.value.ValueLong;
+import org.h2.value.ValueShort;
+import org.h2.value.ValueString;
+import org.h2.value.ValueTime;
+import org.h2.value.ValueTimestamp;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Statistic related simple tests.
+ */
+public class StatisticsPlannerTest extends AbstractPlannerTest {

Review comment:
       Is it still WIP?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to