You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2020/12/30 09:38:14 UTC

[GitHub] [ignite] korlov42 commented on a change in pull request #8596: IGNITE-13545 Index spool

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/IgniteTableImpl.java
##########
@@ -122,8 +122,8 @@ public IgniteTableImpl(TableDescriptor desc) {
     }
 
     /** {@inheritDoc} */
-    @Override public CollocationGroup colocationGroup(PlanningContext ctx) {

Review comment:
       colocation was right because it just 'location' with prefix 'co-'

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.exec;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.query.GridIndex;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.lang.GridIteratorAdapter;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Abstract index scan.
+ */
+public abstract class AbstractIndexScan<Row, IdxRow> implements Iterable<Row>, AutoCloseable {
+    /** */
+    private final GridIndex<IdxRow> idx;
+
+    /** Additional filters. */
+    private final Predicate<Row> filters;
+
+    /** Lower index scan bound. */
+    private final Supplier<Row> lowerBound;
+
+    /** Upper index scan bound. */
+    private final Supplier<Row> upperBound;
+
+    /** */
+    private final Function<Row, Row> rowTransformer;
+
+    /** */
+    protected final ExecutionContext<Row> ectx;
+
+    /** */
+    protected final RelDataType rowType;
+
+    /**
+     * @param ectx Execution context.
+     * @param idx Physical index.
+     * @param filters Additional filters.
+     * @param lowerBound Lower index scan bound.
+     * @param upperBound Upper index scan bound.
+     */
+    public AbstractIndexScan(

Review comment:
       ```suggestion
       protected AbstractIndexScan(
   ```

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractRuntimeSortedIndex.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.exec;
+
+import java.util.Comparator;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.query.GridIndex;
+
+/**
+ * Runtime sorted index.
+ */
+public abstract class AbstractRuntimeSortedIndex<Row> implements GridIndex<Row>, AutoCloseable {
+    /** */
+    protected final ExecutionContext<Row> ectx;
+
+    /** */
+    protected final Comparator<Row> comp;
+
+    /** */
+    protected AbstractRuntimeSortedIndex(
+        ExecutionContext<Row> ectx,
+        Comparator<Row> comp
+    ) {
+        this.ectx = ectx;
+        this.comp = comp;
+    }
+
+    /** */
+    public abstract void push(Row r);
+

Review comment:
       please give a few words about what `push` and `scan` methods are supposed to do

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractRuntimeSortedIndex.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.exec;
+
+import java.util.Comparator;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.query.GridIndex;
+
+/**
+ * Runtime sorted index.
+ */
+public abstract class AbstractRuntimeSortedIndex<Row> implements GridIndex<Row>, AutoCloseable {

Review comment:
       for now we have only one implementation, hence may be it worth to not to spawn a new classes

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractRuntimeSortedIndex.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.exec;
+
+import java.util.Comparator;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.query.GridIndex;
+
+/**
+ * Runtime sorted index.
+ */
+public abstract class AbstractRuntimeSortedIndex<Row> implements GridIndex<Row>, AutoCloseable {
+    /** */
+    protected final ExecutionContext<Row> ectx;
+
+    /** */
+    protected final Comparator<Row> comp;
+
+    /** */
+    protected AbstractRuntimeSortedIndex(
+        ExecutionContext<Row> ectx,
+        Comparator<Row> comp
+    ) {
+        this.ectx = ectx;
+        this.comp = comp;
+    }
+
+    /** */
+    public abstract void push(Row r);
+
+    /** */
+    public Iterable<Row> scan(
+        ExecutionContext<Row> ectx,
+        RelDataType rowType,
+        GridIndex<Row> idx,
+        Predicate<Row> filters,
+        Supplier<Row> lowerBound,
+        Supplier<Row> upperBound
+    ) {
+        return new IndexScan(ectx, rowType, idx, filters, lowerBound, upperBound);
+    }

Review comment:
       ```suggestion
       public Iterable<Row> scan(
           RelDataType rowType,
           Predicate<Row> filters,
           Supplier<Row> lowerBound,
           Supplier<Row> upperBound
       ) {
           return new IndexScan(ectx, rowType, this, filters, lowerBound, upperBound);
       }
   ```
   may be `rowType`should be a class member as well

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractRuntimeSortedIndex.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.exec;
+
+import java.util.Comparator;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.query.GridIndex;
+
+/**
+ * Runtime sorted index.
+ */
+public abstract class AbstractRuntimeSortedIndex<Row> implements GridIndex<Row>, AutoCloseable {
+    /** */
+    protected final ExecutionContext<Row> ectx;
+
+    /** */
+    protected final Comparator<Row> comp;
+
+    /** */
+    protected AbstractRuntimeSortedIndex(
+        ExecutionContext<Row> ectx,
+        Comparator<Row> comp
+    ) {
+        this.ectx = ectx;
+        this.comp = comp;
+    }
+
+    /** */
+    public abstract void push(Row r);
+
+    /** */
+    public Iterable<Row> scan(
+        ExecutionContext<Row> ectx,
+        RelDataType rowType,
+        GridIndex<Row> idx,
+        Predicate<Row> filters,
+        Supplier<Row> lowerBound,
+        Supplier<Row> upperBound
+    ) {
+        return new IndexScan(ectx, rowType, idx, filters, lowerBound, upperBound);
+    }
+
+    /**
+     *
+     */
+    class IndexScan extends AbstractIndexScan<Row, Row> {

Review comment:
       ```suggestion
       private class IndexScan extends AbstractIndexScan<Row, Row> {
   ```

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> extends AbstractRuntimeSortedIndex<Row> {
+    /** Collation. */
+    private final RelCollation collation;
+
+    /** Rows. */
+    private TreeMap<Row, List<Row>> rows;
+
+    /**
+     *
+     */
+    public RuntimeTreeIndex(
+        ExecutionContext<Row> ectx,
+        RelCollation collation,
+        Comparator<Row> comp
+    ) {
+        super(ectx, comp);
+
+        assert Objects.nonNull(collation);
+
+        this.collation = collation;
+        rows = new TreeMap<>(comp);
+    }
+
+    /** */
+    public void push(Row r) {
+        List<Row> eqRows = rows.putIfAbsent(r, new ArrayList<>(Collections.singletonList(r)));
+
+        if (eqRows != null)
+            eqRows.add(r);
+    }
+
+    /** */

Review comment:
       ```suggestion
       /** {@inheritDoc} */
   ```

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> extends AbstractRuntimeSortedIndex<Row> {
+    /** Collation. */
+    private final RelCollation collation;
+
+    /** Rows. */
+    private TreeMap<Row, List<Row>> rows;
+
+    /**
+     *
+     */
+    public RuntimeTreeIndex(
+        ExecutionContext<Row> ectx,
+        RelCollation collation,
+        Comparator<Row> comp
+    ) {
+        super(ectx, comp);
+
+        assert Objects.nonNull(collation);
+
+        this.collation = collation;
+        rows = new TreeMap<>(comp);
+    }
+
+    /** */
+    public void push(Row r) {
+        List<Row> eqRows = rows.putIfAbsent(r, new ArrayList<>(Collections.singletonList(r)));
+
+        if (eqRows != null)
+            eqRows.add(r);
+    }
+
+    /** */
+    @Override public void close() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<Row> find(Row lower, Row upper, BPlusTree.TreeRowClosure<Row, Row> filterC) {
+        return find(lower, upper);
+    }
+
+    /** */
+    private GridCursor<Row> find(Row lower, Row upper) {
+        int firstCol = F.first(collation.getKeys());
+
+        if (ectx.rowHandler().get(firstCol, lower) != null && ectx.rowHandler().get(firstCol, upper) != null)
+            return new Cursor(rows.subMap(lower, true, upper, true));
+        else if (ectx.rowHandler().get(firstCol, lower) == null && ectx.rowHandler().get(firstCol, upper) != null)
+            return new Cursor(rows.headMap(upper, true));
+        else if (ectx.rowHandler().get(firstCol, lower) != null && ectx.rowHandler().get(firstCol, upper) == null)
+            return new Cursor(rows.tailMap(lower, true));
+        else
+            return new Cursor(rows);
+    }
+
+    /**
+     *
+     */
+    private class Cursor implements GridCursor<Row> {
+        /** Sub map. */

Review comment:
       wrong javadoc here and below

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> extends AbstractRuntimeSortedIndex<Row> {
+    /** Collation. */
+    private final RelCollation collation;
+
+    /** Rows. */
+    private TreeMap<Row, List<Row>> rows;
+
+    /**
+     *
+     */
+    public RuntimeTreeIndex(
+        ExecutionContext<Row> ectx,
+        RelCollation collation,
+        Comparator<Row> comp
+    ) {
+        super(ectx, comp);
+
+        assert Objects.nonNull(collation);
+
+        this.collation = collation;
+        rows = new TreeMap<>(comp);
+    }
+
+    /** */
+    public void push(Row r) {
+        List<Row> eqRows = rows.putIfAbsent(r, new ArrayList<>(Collections.singletonList(r)));
+
+        if (eqRows != null)
+            eqRows.add(r);
+    }
+
+    /** */
+    @Override public void close() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<Row> find(Row lower, Row upper, BPlusTree.TreeRowClosure<Row, Row> filterC) {
+        return find(lower, upper);

Review comment:
       ```suggestion
       @Override public GridCursor<Row> find(Row lower, Row upper, BPlusTree.TreeRowClosure<Row, Row> filterC) {
           assert filterC == null;
           
           return find(lower, upper);
   ```

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> extends AbstractRuntimeSortedIndex<Row> {
+    /** Collation. */
+    private final RelCollation collation;
+
+    /** Rows. */
+    private TreeMap<Row, List<Row>> rows;
+
+    /**
+     *
+     */
+    public RuntimeTreeIndex(
+        ExecutionContext<Row> ectx,
+        RelCollation collation,
+        Comparator<Row> comp
+    ) {
+        super(ectx, comp);
+
+        assert Objects.nonNull(collation);
+
+        this.collation = collation;
+        rows = new TreeMap<>(comp);
+    }
+
+    /** */
+    public void push(Row r) {
+        List<Row> eqRows = rows.putIfAbsent(r, new ArrayList<>(Collections.singletonList(r)));
+
+        if (eqRows != null)
+            eqRows.add(r);
+    }
+
+    /** */
+    @Override public void close() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<Row> find(Row lower, Row upper, BPlusTree.TreeRowClosure<Row, Row> filterC) {
+        return find(lower, upper);
+    }
+
+    /** */
+    private GridCursor<Row> find(Row lower, Row upper) {
+        int firstCol = F.first(collation.getKeys());
+
+        if (ectx.rowHandler().get(firstCol, lower) != null && ectx.rowHandler().get(firstCol, upper) != null)
+            return new Cursor(rows.subMap(lower, true, upper, true));
+        else if (ectx.rowHandler().get(firstCol, lower) == null && ectx.rowHandler().get(firstCol, upper) != null)
+            return new Cursor(rows.headMap(upper, true));
+        else if (ectx.rowHandler().get(firstCol, lower) != null && ectx.rowHandler().get(firstCol, upper) == null)
+            return new Cursor(rows.tailMap(lower, true));
+        else
+            return new Cursor(rows);
+    }
+
+    /**
+     *
+     */
+    private class Cursor implements GridCursor<Row> {
+        /** Sub map. */
+        private final Iterator<Map.Entry<Row, List<Row>>> mapIt;
+
+        /** Sub map. */
+        private Iterator<Row> listIt;
+
+        /** */
+        private Row row;
+
+        /** */
+        public Cursor(SortedMap<Row, List<Row>> subMap) {
+            mapIt = subMap.entrySet().iterator();
+            listIt = null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean next() throws IgniteCheckedException {
+            if (!hasNext())
+                return false;
+
+            next0();
+
+            return true;
+        }
+
+        /** */
+        private boolean hasNext() {
+            return listIt != null && listIt.hasNext() || mapIt.hasNext();
+        }
+
+        /** */
+        private void next0() {
+            if (listIt != null && listIt.hasNext())
+                row = listIt.next();
+            else {
+                listIt = mapIt.next().getValue().iterator();
+
+                row = listIt.next();
+            }
+        }

Review comment:
       ```suggestion
           private void next0() {
               if (listIt == null || !listIt.hasNext())
                   listIt = mapIt.next().getValue().iterator();
   
               row = listIt.next();
           }
   ```

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> extends AbstractRuntimeSortedIndex<Row> {
+    /** Collation. */
+    private final RelCollation collation;
+
+    /** Rows. */
+    private TreeMap<Row, List<Row>> rows;
+
+    /**
+     *
+     */
+    public RuntimeTreeIndex(
+        ExecutionContext<Row> ectx,
+        RelCollation collation,
+        Comparator<Row> comp
+    ) {
+        super(ectx, comp);
+
+        assert Objects.nonNull(collation);
+
+        this.collation = collation;
+        rows = new TreeMap<>(comp);
+    }
+
+    /** */
+    public void push(Row r) {
+        List<Row> eqRows = rows.putIfAbsent(r, new ArrayList<>(Collections.singletonList(r)));
+
+        if (eqRows != null)
+            eqRows.add(r);
+    }

Review comment:
       ```suggestion
       /** {@inheritDoc} */
       @Override public void push(Row r) {
           rows.computeIfAbsent(r, k -> new ArrayList<>()).add(r);
       }
   ```

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> extends AbstractRuntimeSortedIndex<Row> {
+    /** Collation. */
+    private final RelCollation collation;
+
+    /** Rows. */
+    private TreeMap<Row, List<Row>> rows;
+
+    /**
+     *
+     */
+    public RuntimeTreeIndex(
+        ExecutionContext<Row> ectx,
+        RelCollation collation,
+        Comparator<Row> comp
+    ) {
+        super(ectx, comp);
+
+        assert Objects.nonNull(collation);
+
+        this.collation = collation;
+        rows = new TreeMap<>(comp);
+    }
+
+    /** */
+    public void push(Row r) {
+        List<Row> eqRows = rows.putIfAbsent(r, new ArrayList<>(Collections.singletonList(r)));
+
+        if (eqRows != null)
+            eqRows.add(r);
+    }
+
+    /** */
+    @Override public void close() throws Exception {
+        // No-op.

Review comment:
       Do you now where this method will be invoked? May be we should to clear the map here?

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;

Review comment:
       ```suggestion
   import java.util.Comparator;
   ```

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> extends AbstractRuntimeSortedIndex<Row> {
+    /** Collation. */
+    private final RelCollation collation;
+
+    /** Rows. */
+    private TreeMap<Row, List<Row>> rows;

Review comment:
       ```suggestion
       private final TreeMap<Row, List<Row>> rows;
   ```

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> extends AbstractRuntimeSortedIndex<Row> {
+    /** Collation. */
+    private final RelCollation collation;
+
+    /** Rows. */
+    private TreeMap<Row, List<Row>> rows;
+
+    /**
+     *
+     */
+    public RuntimeTreeIndex(
+        ExecutionContext<Row> ectx,
+        RelCollation collation,
+        Comparator<Row> comp
+    ) {
+        super(ectx, comp);
+
+        assert Objects.nonNull(collation);
+
+        this.collation = collation;
+        rows = new TreeMap<>(comp);
+    }
+
+    /** */
+    public void push(Row r) {
+        List<Row> eqRows = rows.putIfAbsent(r, new ArrayList<>(Collections.singletonList(r)));
+
+        if (eqRows != null)
+            eqRows.add(r);
+    }
+
+    /** */
+    @Override public void close() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<Row> find(Row lower, Row upper, BPlusTree.TreeRowClosure<Row, Row> filterC) {
+        return find(lower, upper);
+    }
+
+    /** */
+    private GridCursor<Row> find(Row lower, Row upper) {

Review comment:
       is it possible when `lower` is bigger than `upper`? #subMap will throw an Exception in such case




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org