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/11/24 14:54:46 UTC

[GitHub] [ignite] timoninmaxim opened a new pull request #8490: WIP

timoninmaxim opened a new pull request #8490:
URL: https://github.com/apache/ignite/pull/8490


   Moving indexes from indexing to core module.


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r582071307



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexSchema.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.h2.index;
+
+import org.apache.ignite.cache.query.index.sorted.NullsOrder;
+import org.apache.ignite.cache.query.index.sorted.Order;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.JavaObjectKey;
+import org.apache.ignite.internal.cache.query.index.sorted.NullKey;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexKeyTypes;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.h2.table.IndexColumn;
+
+/**
+ * Schema for QueryIndex.
+ */
+public class QueryIndexSchema implements SortedIndexSchema {
+    /** Key definitions. */
+    private final IndexKeyDefinition[] idxKeyDefinitions;
+
+    /** H2 index columns. */
+    private final IndexColumn[] h2IdxColumns;
+
+    /** Cache descriptor. */
+    private final GridH2RowDescriptor cacheDesc;
+
+    /** Table. */
+    private final GridH2Table table;
+
+    /** */
+    public QueryIndexSchema(GridH2Table table, IndexColumn[] h2IndexColumns) {
+        this.table = table;
+
+        cacheDesc = table.rowDescriptor();
+
+        idxKeyDefinitions = new IndexKeyDefinition[h2IndexColumns.length];
+
+        this.h2IdxColumns = h2IndexColumns.clone();
+
+        for (int i = 0; i < h2IndexColumns.length; ++i)
+            addKeyDefinition(i, h2IndexColumns[i]);
+
+        IndexColumn.mapColumns(h2IndexColumns, table);
+    }
+
+    /** */
+    private void addKeyDefinition(int i, IndexColumn c) {
+        GridQueryTypeDescriptor type = cacheDesc.type();
+
+        Class<?> idxKeyCls;
+
+        int colId = c.column.getColumnId();
+
+        if (cacheDesc.isKeyColumn(colId) || cacheDesc.isKeyAliasColumn(colId))
+            idxKeyCls = type.keyClass();
+        else if (cacheDesc.isValueColumn(colId) || cacheDesc.isKeyAliasColumn(colId))
+            idxKeyCls = type.valueClass();
+        else
+            idxKeyCls = type.property(c.columnName).type();
+
+        idxKeyDefinitions[i] = new IndexKeyDefinition(
+            c.columnName, c.column.getType(), idxKeyCls, getSortOrder(c.sortType));
+    }
+
+    /** Maps H2 column order to Ignite index order. */
+    private Order getSortOrder(int sortType) {
+        SortOrder sortOrder = (sortType & 1) != 0 ? SortOrder.DESC : SortOrder.ASC;
+
+        NullsOrder nullsOrder = (sortType & 2) != 0 ? NullsOrder.NULLS_FIRST : NullsOrder.NULLS_LAST;
+
+        return new Order(sortOrder, nullsOrder);
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public IndexKeyDefinition[] getKeyDefinitions() {
+        return idxKeyDefinitions.clone();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getIndexKey(int idx, CacheDataRow row) {
+        Object o = getKey(idx, row);
+
+        if (o == null)
+            return NullKey.INSTANCE;
+
+        if (idxKeyDefinitions[idx].getIdxType() == IndexKeyTypes.JAVA_OBJECT)
+            return new JavaObjectKey(o);
+
+        return o;
+    }
+
+    /** */
+    private Object getKey(int idx, CacheDataRow row) {
+        int cacheIdx = h2IdxColumns[idx].column.getColumnId();
+
+        switch (cacheIdx) {
+            case QueryUtils.KEY_COL:
+                return key(row);
+
+            case QueryUtils.VAL_COL:
+                return value(row);
+
+            default:
+                if (cacheDesc.isKeyAliasColumn(cacheIdx))
+                    return key(row);
+
+                else if (cacheDesc.isValueAliasColumn(cacheIdx))
+                    return value(row);
+
+                // columnValue ignores default columns (_KEY, _VAL), so make this shift.
+                return cacheDesc.columnValue(row.key(), row.value(), cacheIdx - QueryUtils.DEFAULT_COLUMNS_COUNT);
+        }
+    }

Review comment:
       Agree. FIxed.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r582074123



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexSchema.java
##########
@@ -15,25 +15,40 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringFixed;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 
 /**
- * Inline index column implementation for inlining strings of fixed length.
+ * Schema for sorted index.
  */
-public class FixedStringInlineIndexColumn extends StringInlineIndexColumn {
+public interface SortedIndexSchema {
     /**
-     * @param col Column.
+     * Describe all index keys.
      */
-    public FixedStringInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_FIXED, useOptimizedCompare, false);
-    }
+    public IndexKeyDefinition[] getKeyDefinitions();

Review comment:
       Index keys moved to definition.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r568740505



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)

Review comment:
       Yes, it looks pretty strange. But it's the same logic as org.h2.index.SingleRowCursor does ([Link](https://github.com/h2database/h2database/blob/version-1.4.197/h2/src/main/org/h2/index/SingleRowCursor.java))
   
   Also we have some tests (or logic) that differently works with SingleCursor. Some of them just use `get`, some of them make `next -> get` 




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r575830101



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/sorted/SortedIndex.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.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface for sorted Ignite indexes.
+ */
+public interface SortedIndex extends Index {
+    /**
+     * Finds index rows by specified range in specifed tree segment. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(@Nullable IndexKey lower, @Nullable IndexKey upper, int segment) throws IgniteCheckedException;
+
+    /**
+     * Finds index rows by specified range in specifed tree segment with cache filtering. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(IndexKey lower, IndexKey upper, int segment, IndexingQueryFilter filter)
+        throws IgniteCheckedException;
+
+    /**
+     * Finds first or last index row for specified tree segment and cache filter.
+     *
+     * @param firstOrLast if {@code true} then return first index row or otherwise last row.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> findFirstOrLast(boolean firstOrLast, int segment, IndexingQueryFilter filter)
+        throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment.
+     *
+     * @param segment Number of tree segment to find.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment with cache filter.
+     *
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment, IndexingQueryFilter filter) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows for all segments.
+     *
+     * @return total count of index rows.
+     */
+    public long totalCount() throws IgniteCheckedException;
+
+    /**
+     * Returns amount of index tree segments.
+     *
+     * @return amount of index tree segments.
+     */
+    public int segmentsCount();

Review comment:
       > it just extends existing one and hardcode segment with 0 value
   
   Extension should brings new features, not limits them. And current approach violates the Interface Segregation Principle, I think




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r567406515



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/Index.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for Ignite indexes.
+ */
+public interface Index {
+    /**
+     * Unique ID.
+     */
+    public UUID id();
+
+    /**
+     * Index name.
+     */
+    public String name();
+
+    /**
+     * Checks whether index handles specified cache row.
+     *
+     * @param row Cache row.
+     * @return Whether index handles specified cache row
+     */
+    public boolean belongsToIndex(CacheDataRow row) throws IgniteCheckedException;

Review comment:
       Personally I'd prefer to rename it to 'indexesRow' or something like this, because right now it sounds like 'Index belongs to index' and row is passed as param. 

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/Index.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for Ignite indexes.
+ */
+public interface Index {
+    /**
+     * Unique ID.
+     */
+    public UUID id();
+
+    /**
+     * Index name.
+     */
+    public String name();
+
+    /**
+     * Checks whether index handles specified cache row.
+     *
+     * @param row Cache row.
+     * @return Whether index handles specified cache row
+     */
+    public boolean belongsToIndex(CacheDataRow row) throws IgniteCheckedException;
+
+    /**
+     * Callback that runs when the underlying cache is updated.
+     *
+     * @param oldRow Cache row that was replaced with newRow.
+     * @param newRow Cache row that was stored.
+     * @param prevRowAvailable Whether oldRow available.
+     */
+    public void onUpdate(@Nullable CacheDataRow oldRow, @Nullable CacheDataRow newRow, boolean prevRowAvailable)
+        throws IgniteCheckedException;
+
+    /**
+     * Put cache row to index.
+     *
+     * @return {@code True} if replaced existing row.
+     */
+    public boolean putx(CacheDataRow row) throws IgniteCheckedException;

Review comment:
       why you decided to add a '-x' suffix to method name? Currently `InlineIndexImpl` has two `putx` methods and no 'put'. I'd prefer to name it just `put`

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/sorted/SortedIndex.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.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface for sorted Ignite indexes.
+ */
+public interface SortedIndex extends Index {
+    /**
+     * Finds index rows by specified range in specifed tree segment. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(@Nullable IndexKey lower, @Nullable IndexKey upper, int segment) throws IgniteCheckedException;
+
+    /**
+     * Finds index rows by specified range in specifed tree segment with cache filtering. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(IndexKey lower, IndexKey upper, int segment, IndexingQueryFilter filter)
+        throws IgniteCheckedException;
+
+    /**
+     * Finds first or last index row for specified tree segment and cache filter.
+     *
+     * @param firstOrLast if {@code true} then return first index row or otherwise last row.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> findFirstOrLast(boolean firstOrLast, int segment, IndexingQueryFilter filter)
+        throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment.
+     *
+     * @param segment Number of tree segment to find.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment with cache filter.
+     *
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment, IndexingQueryFilter filter) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows for all segments.
+     *
+     * @return total count of index rows.
+     */
+    public long totalCount() throws IgniteCheckedException;
+
+    /**
+     * Returns amount of index tree segments.
+     *
+     * @return amount of index tree segments.
+     */
+    public int segmentsCount();

Review comment:
       actually segments has nothing common with a generic sorted index, thus it would be better to introduce one more abstraction like SegmentedIndex

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyDefinition.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.cache.query.index.sorted.Order;
+
+/**
+ * Defines a signle index key.
+ */
+public class IndexKeyDefinition {
+    /** Index key name. */
+    private final String name;
+
+    /** Index key type. @see IndexKeyTypes. */

Review comment:
       please format this like a proper javadoc link

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/sorted/Order.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.cache.query.index.sorted;
+
+/**
+ * Represents ordering of rows within sorted index.
+ */
+public class Order {
+    /** */
+    private NullsOrder nullsOrder;
+
+    /** */
+    private SortOrder sortOrder;

Review comment:
       let's make these fields final

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)

Review comment:
       This should be strong equality, otherwise it violates the "single value" contract.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/IndexName.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.cache.query.index;
+
+import java.io.Serializable;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents list of names that fully describes index domain (schema, cache, table, index).
+ */
+public class IndexName implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Schema name of {@code null} if index is not related to SQL schema. */
+    private final @Nullable String schemaName;
+
+    /** Schema name of {@code null} if index is not related to SQL table. */
+    private final @Nullable String tableName;
+
+    /** Cache name. */
+    private final String cacheName;
+
+    /** Index name. */
+    private final String idxName;
+
+    /** */
+    public IndexName(String cacheName, @Nullable String schemaName, @Nullable String tableName, String idxName) {
+        this.cacheName = cacheName;
+        this.schemaName = schemaName;
+        this.tableName = tableName;
+        this.idxName = idxName;
+    }
+
+    /**
+     * @return FQDN index name.
+     */
+    public String fqdnIdxName() {

Review comment:
       let's rename it to `fullName`

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/sorted/SortedIndex.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.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface for sorted Ignite indexes.
+ */
+public interface SortedIndex extends Index {
+    /**
+     * Finds index rows by specified range in specifed tree segment. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(@Nullable IndexKey lower, @Nullable IndexKey upper, int segment) throws IgniteCheckedException;
+
+    /**
+     * Finds index rows by specified range in specifed tree segment with cache filtering. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(IndexKey lower, IndexKey upper, int segment, IndexingQueryFilter filter)
+        throws IgniteCheckedException;
+
+    /**
+     * Finds first or last index row for specified tree segment and cache filter.
+     *
+     * @param firstOrLast if {@code true} then return first index row or otherwise last row.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> findFirstOrLast(boolean firstOrLast, int segment, IndexingQueryFilter filter)

Review comment:
       let's split it on two separate methods

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/IndexName.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.cache.query.index;
+
+import java.io.Serializable;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents list of names that fully describes index domain (schema, cache, table, index).
+ */
+public class IndexName implements Serializable {

Review comment:
       Looks kinda cumbersome.
   Actually a tuple (schemaName, indexName) describes an index uniquely. Table name has nothing common with indexName. And for non-SQL indexes a cache name could be used for schemaName

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/sorted/IndexKey.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents a complex index key.
+ */
+public interface IndexKey {

Review comment:
       The purpose of this entity is unclear to me. Right now it's just a straitforward replacement of H2's SearchRow, but then it should not have a link to CacheDataRow

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
##########
@@ -15,25 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.cache.query.index;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringIgnoreCase;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static java.lang.Boolean.FALSE;
+import static java.lang.Boolean.TRUE;
 
 /**
- * Inline index column implementation for inlining strings ignore case.
+ * Abstract class for all Index implementations.
  */
-public class StringIgnoreCaseInlineIndexColumn extends StringInlineIndexColumn {
+public abstract class AbstractIndex implements Index {

Review comment:
       Don't think we need this abstraction right now because only one index implementation extends this. 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get() {
+            return null;

Review comment:
       it's better to throw exception here

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexSchema.java
##########
@@ -15,25 +15,40 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringFixed;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 
 /**
- * Inline index column implementation for inlining strings of fixed length.
+ * Schema for sorted index.
  */
-public class FixedStringInlineIndexColumn extends StringInlineIndexColumn {
+public interface SortedIndexSchema {
     /**
-     * @param col Column.
+     * Describe all index keys.
      */
-    public FixedStringInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_FIXED, useOptimizedCompare, false);
-    }
+    public IndexKeyDefinition[] getKeyDefinitions();

Review comment:
       IndexKeyDefinition array should be part of IndexDefinition. Other fields have nothing common with scheme of a sorted index




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r582086425



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
##########
@@ -464,41 +427,59 @@ public GridKernalContext kernalContext() {
      * @param unwrappedCols Unwrapped index columns for complex types.
      * @param wrappedCols Index columns as is complex types.
      * @param inlineSize Index inline size.
+     * @param cacheVisitor whether index created with new cache or on existing one.
      * @return Index.
      */
     @SuppressWarnings("ConstantConditions")
     GridH2IndexBase createSortedIndex(String name, GridH2Table tbl, boolean pk, boolean affinityKey,
-        List<IndexColumn> unwrappedCols, List<IndexColumn> wrappedCols, int inlineSize) {
-        try {
-            GridCacheContextInfo cacheInfo = tbl.cacheInfo();
-
-            if (log.isDebugEnabled())
-                log.debug("Creating cache index [cacheId=" + cacheInfo.cacheId() + ", idxName=" + name + ']');
-
-            if (cacheInfo.affinityNode()) {
-                final int segments = tbl.rowDescriptor().context().config().getQueryParallelism();
-
-                H2RowCache cache = rowCache.forGroup(cacheInfo.groupId());
-
-                return H2TreeIndex.createIndex(
-                    cacheInfo.cacheContext(),
-                    cache,
-                    tbl,
-                    name,
-                    pk,
-                    affinityKey,
-                    unwrappedCols,
-                    wrappedCols,
-                    inlineSize,
-                    segments,
-                    log
-                );
-            }
+        List<IndexColumn> unwrappedCols, List<IndexColumn> wrappedCols, int inlineSize, @Nullable SchemaIndexCacheVisitor cacheVisitor) {
+        GridCacheContextInfo cacheInfo = tbl.cacheInfo();
+
+        if (log.isDebugEnabled())
+            log.debug("Creating cache index [cacheId=" + cacheInfo.cacheId() + ", idxName=" + name + ']');
+
+        QueryIndexSchema schemaUnwrapped = new QueryIndexSchema(
+            tbl, unwrappedCols.toArray(new IndexColumn[0]));
+
+        QueryIndexSchema schemaWrapped = new QueryIndexSchema(
+            tbl, wrappedCols.toArray(new IndexColumn[0]));
+
+        if (cacheInfo.affinityNode()) {
+            SortedIndexDefinition idxDef = new QueryIndexDefinition(

Review comment:
       Wait for merge of [#8811](https://github.com/apache/ignite/pull/8811).




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r586618568



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get() {
+            return null;

Review comment:
       Unfortunately I can't fix this. H2Cursor for queries like "select MAX(col) from table" relies on this null. The same issue with SingleCursor. I can add comment there, and create a ticket to have this in mind for Calcite integration. WDYT?




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r568703377



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/Index.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for Ignite indexes.
+ */
+public interface Index {
+    /**
+     * Unique ID.
+     */
+    public UUID id();
+
+    /**
+     * Index name.
+     */
+    public String name();
+
+    /**
+     * Checks whether index handles specified cache row.
+     *
+     * @param row Cache row.
+     * @return Whether index handles specified cache row
+     */
+    public boolean belongsToIndex(CacheDataRow row) throws IgniteCheckedException;
+
+    /**
+     * Callback that runs when the underlying cache is updated.
+     *
+     * @param oldRow Cache row that was replaced with newRow.
+     * @param newRow Cache row that was stored.
+     * @param prevRowAvailable Whether oldRow available.
+     */
+    public void onUpdate(@Nullable CacheDataRow oldRow, @Nullable CacheDataRow newRow, boolean prevRowAvailable)
+        throws IgniteCheckedException;
+
+    /**
+     * Put cache row to index.
+     *
+     * @return {@code True} if replaced existing row.
+     */
+    public boolean putx(CacheDataRow row) throws IgniteCheckedException;

Review comment:
       The previous implementation of H2TreeIndex used the `putx` method only. It did have the `put` method, but in all places it's not used returned value, just checks it for existing, so it's similar to `putx` logic. So it looks like it's just not required. I can add it to make API more wide, but as it won't be a part of public for a while API, do we need it?
   
   




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r575805700



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyImpl.java
##########
@@ -15,41 +15,48 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
-import org.apache.ignite.internal.pagemem.PageUtils;
+import java.util.Arrays;
+import org.apache.ignite.cache.query.index.sorted.IndexKey;
 
 /**
- *
+ * Complex index key that represents a user index query.
  */
-class H2MvccExtrasInnerIO extends AbstractH2ExtrasInnerIO {
-    /**
-     * @param type Page type.
-     * @param ver Page format version.
-     * @param payloadSize Payload size.
-     */
-    H2MvccExtrasInnerIO(short type, int ver, int payloadSize) {
-        super(type, ver, 28, payloadSize);
+public class IndexKeyImpl implements IndexKey {

Review comment:
       is it used anywhere?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexSchema.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.h2.index;
+
+import org.apache.ignite.cache.query.index.sorted.NullsOrder;
+import org.apache.ignite.cache.query.index.sorted.Order;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.JavaObjectKey;
+import org.apache.ignite.internal.cache.query.index.sorted.NullKey;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexKeyTypes;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.h2.table.IndexColumn;
+
+/**
+ * Schema for QueryIndex.
+ */
+public class QueryIndexSchema implements SortedIndexSchema {
+    /** Key definitions. */
+    private final IndexKeyDefinition[] idxKeyDefinitions;
+
+    /** H2 index columns. */
+    private final IndexColumn[] h2IdxColumns;
+
+    /** Cache descriptor. */
+    private final GridH2RowDescriptor cacheDesc;
+
+    /** Table. */
+    private final GridH2Table table;
+
+    /** */
+    public QueryIndexSchema(GridH2Table table, IndexColumn[] h2IndexColumns) {
+        this.table = table;
+
+        cacheDesc = table.rowDescriptor();
+
+        idxKeyDefinitions = new IndexKeyDefinition[h2IndexColumns.length];
+
+        this.h2IdxColumns = h2IndexColumns.clone();
+
+        for (int i = 0; i < h2IndexColumns.length; ++i)
+            addKeyDefinition(i, h2IndexColumns[i]);
+
+        IndexColumn.mapColumns(h2IndexColumns, table);
+    }
+
+    /** */
+    private void addKeyDefinition(int i, IndexColumn c) {
+        GridQueryTypeDescriptor type = cacheDesc.type();
+
+        Class<?> idxKeyCls;
+
+        int colId = c.column.getColumnId();
+
+        if (cacheDesc.isKeyColumn(colId) || cacheDesc.isKeyAliasColumn(colId))
+            idxKeyCls = type.keyClass();
+        else if (cacheDesc.isValueColumn(colId) || cacheDesc.isKeyAliasColumn(colId))
+            idxKeyCls = type.valueClass();
+        else
+            idxKeyCls = type.property(c.columnName).type();
+
+        idxKeyDefinitions[i] = new IndexKeyDefinition(
+            c.columnName, c.column.getType(), idxKeyCls, getSortOrder(c.sortType));
+    }
+
+    /** Maps H2 column order to Ignite index order. */
+    private Order getSortOrder(int sortType) {
+        SortOrder sortOrder = (sortType & 1) != 0 ? SortOrder.DESC : SortOrder.ASC;
+
+        NullsOrder nullsOrder = (sortType & 2) != 0 ? NullsOrder.NULLS_FIRST : NullsOrder.NULLS_LAST;
+
+        return new Order(sortOrder, nullsOrder);
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public IndexKeyDefinition[] getKeyDefinitions() {
+        return idxKeyDefinitions.clone();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getIndexKey(int idx, CacheDataRow row) {
+        Object o = getKey(idx, row);
+
+        if (o == null)
+            return NullKey.INSTANCE;
+
+        if (idxKeyDefinitions[idx].getIdxType() == IndexKeyTypes.JAVA_OBJECT)
+            return new JavaObjectKey(o);
+
+        return o;
+    }
+
+    /** */
+    private Object getKey(int idx, CacheDataRow row) {
+        int cacheIdx = h2IdxColumns[idx].column.getColumnId();
+
+        switch (cacheIdx) {
+            case QueryUtils.KEY_COL:
+                return key(row);
+
+            case QueryUtils.VAL_COL:
+                return value(row);
+
+            default:
+                if (cacheDesc.isKeyAliasColumn(cacheIdx))
+                    return key(row);
+
+                else if (cacheDesc.isValueAliasColumn(cacheIdx))
+                    return value(row);
+
+                // columnValue ignores default columns (_KEY, _VAL), so make this shift.
+                return cacheDesc.columnValue(row.key(), row.value(), cacheIdx - QueryUtils.DEFAULT_COLUMNS_COUNT);
+        }
+    }

Review comment:
       ```suggestion
       private Object getKey(int idx, CacheDataRow row) {
           int cacheIdx = h2IdxColumns[idx].column.getColumnId();
   
           if (cacheDesc.isKeyColumn(cacheIdx))
               return key(row);
   
           else if (cacheDesc.isValueColumn(cacheIdx))
               return value(row);
   
           // columnValue ignores default columns (_KEY, _VAL), so make this shift.
           return cacheDesc.columnValue(row.key(), row.value(), cacheIdx - QueryUtils.DEFAULT_COLUMNS_COUNT);
       }
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: [WIP] IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r548503107



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
##########
@@ -15,25 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;

Review comment:
       Actually I don't know why GitHub creates the link StringIgnoreCaseInlineIndexColumn -> AbstractIndex. `git log` for AbstractIndex is the only commit. I tried to git mv, git rm, create file from scratch again. But nothing helps and GitHub still show this again. 
   
   The same time Intellij IDEA shows history of this file differently, it saw it as GridQueryRowCacheCleaner -> AbstractIndex.
   
   Row git log is:
   
   ```
   % git log modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
   commit cee7855760b3fa1867a9e44e52be4ef786f108ad
   Author: Maksim Timonin <ti...@gmail.com>
   Date:   Thu Dec 10 01:25:18 2020 +0300
   
       Rebuild indexes, fix tests
   ```
   
   If somebody have any idea how to fix this, please share this.
   




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r579626874



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/ThreadLocalSchemaHolder.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.cache.query.index.sorted.inline.io;
+
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+
+/**
+ * Holds an index schema during work session with an index tree.
+ */
+public class ThreadLocalSchemaHolder {
+    /** */
+    private static final ThreadLocal<SortedIndexSchema> holder = new ThreadLocal<>();

Review comment:
       Agree. Setup thread local with holder.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: [WIP] IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r548503107



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
##########
@@ -15,25 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;

Review comment:
       Actually I don't know why GitHub creates the link StringIgnoreCaseInlineIndexColumn -> AbstractIndex. `git log` for AbstractIndex is the only commit. I tried to git mv, git rm, create file from scratch again. But nothing helps and GitHub still show this again. 
   
   The same time Intellij IDEA shows history of this file differently, it saw it as GridQueryRowCacheCleaner -> AbstractIndex.
   
   ```
   % git log modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
   commit cee7855760b3fa1867a9e44e52be4ef786f108ad
   Author: Maksim Timonin <ti...@gmail.com>
   Date:   Thu Dec 10 01:25:18 2020 +0300
   
       Rebuild indexes, fix tests
   ```
   
   If somebody have any idea how to fix this, please share this.
   




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r568851658



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get() {
+            return null;

Review comment:
       It's implemented the way it was before. Please check GridH2Cursor.EMPTY [Link](https://github.com/apache/ignite/blob/2d2044a63440e9b9d8495f35819ccee26d8324e4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Cursor.java#L31)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get() {
+            return null;

Review comment:
       It's implemented the way it was before. Please check GridH2Cursor.EMPTY ([Link](https://github.com/apache/ignite/blob/2d2044a63440e9b9d8495f35819ccee26d8324e4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Cursor.java#L31))




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r586469276



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)

Review comment:
       I've run tests again, for queries like "select MAX(col) from table" H2 just use get() without next(). So this is a reason to implement it this way. I can add a note about in javadoc about it and create a bug in Jira for futher development of Calcite. Alternative is create a new interface of Cursor. I'd like to stay with GridCursor interface. WDYT? 

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)

Review comment:
       I've run tests again, for queries like "select MAX(col) from table" H2 just use get() without next(). So this is a reason to implement it this way. I can add a note about in javadoc about it and create a bug in Jira for further development of Calcite. Alternative is create a new interface of Cursor. I'd like to stay with GridCursor interface. WDYT? 




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r575831583



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/DefaultIndexRowComparator.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * Default row comparator. Consider that every index key extends Comparable interface.
+ * Does not support comparation of different key types.
+ */
+public class DefaultIndexRowComparator implements IndexRowComparator {

Review comment:
       is it used anywhere?




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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r601276598



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateValueUtils.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+/**
+ * DateValue is a representation of a date in bit form:
+ *
+ * dv = (year << SHIFT_YEAR) | (month << SHIFT_MONTH) | day.
+ */
+public class DateValueUtils {

Review comment:
       Should we use interface to hold constants or add private constructor? 




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r601295792



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedSegmentedIndex.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface for sorted and segmented Ignite indexes.
+ */
+public interface SortedSegmentedIndex extends Index {
+    /**
+     * Finds index rows by specified range in specifed tree segment. Range can be bound or unbound.

Review comment:
       fixed

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cache.query.index;
+
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Base interface for Ignite index factories.
+ */
+public interface IndexFactory {
+    /**
+     * Creates index by specifed defition for specified cache.

Review comment:
       fixed

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */

Review comment:
       fixed

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/defragmentation/DefragIndexFactory.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.cache.query.index.sorted.defragmentation;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandlerFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.MetaPageInfo;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineRecommender;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.RootPage;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+
+/**
+ * Creates temporary index to defragment old index.
+ */
+public class DefragIndexFactory extends InlineIndexFactory {
+    /** Temporary offheap manager. */
+    private final IgniteCacheOffheapManager offheap;
+
+    /** Old index. */
+    private final InlineIndex oldIdx;
+
+    /** Temporary cache page memory. */
+    private final PageMemory newCachePageMemory;
+
+    /** */
+    private final InlineIndexRowHandlerFactory rowHndFactory;
+
+    /** */
+    public DefragIndexFactory(IgniteCacheOffheapManager offheap, PageMemory newCachePageMemory, InlineIndex oldIdx) {
+        // Row handler factory that produces no-op handler.
+        rowHndFactory = (def, settings) -> oldIdx.segment(0).rowHandler();
+
+        this.offheap = offheap;
+        this.oldIdx = oldIdx;
+        this.newCachePageMemory = newCachePageMemory;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected InlineIndexTree createIndexSegment(GridCacheContext<?, ?> cctx, SortedIndexDefinition def,
+        RootPage rootPage, IoStatisticsHolder stats, InlineRecommender recommender, int segmentNum) throws Exception {
+
+        InlineIndexTree tree = new InlineIndexTree(
+            def,
+            cctx,
+            def.treeName(),
+            offheap,
+            offheap.reuseListForIndex(def.treeName()),
+            newCachePageMemory,
+            // Use old row handler to have access to inline index key types.
+            pageIoResolver(),
+            rootPage.pageId().pageId(),
+            rootPage.isAllocated(),
+            oldIdx.inlineSize(),
+            def.keyTypeSettings(),
+            null,
+            stats,
+            rowHndFactory,
+            null
+        );
+
+        final MetaPageInfo oldInfo = oldIdx.segment(segmentNum).metaInfo();
+
+        tree.copyMetaInfo(oldInfo);
+
+        tree.enableSequentialWriteMode();
+
+        return tree;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RootPage rootPage(GridCacheContext<?, ?> ctx, String treeName, int segment) throws Exception {
+        return offheap.rootPageForIndex(ctx.cacheId(), treeName, segment);
+    }
+
+    /** */
+    private PageIoResolver pageIoResolver() {
+        return pageAddr -> {
+            PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+            if (io instanceof BPlusMetaIO)
+                return io;
+
+            //noinspection unchecked,rawtypes,rawtypes
+            return wrap((BPlusIO)io, rowHndFactory.create(null, null));
+        };
+    }
+
+    /** */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    static BPlusIO<IndexRow> wrap(BPlusIO<IndexRow> io, InlineIndexRowHandler rowHnd) {
+        assert io instanceof InlineIO;
+
+        if (io instanceof BPlusInnerIO) {
+            assert io instanceof AbstractInlineInnerIO
+                || io instanceof InlineInnerIO;
+
+            return new BPlusInnerIoDelegate((BPlusInnerIO<IndexRow>)io, rowHnd);
+        }
+        else {
+            assert io instanceof AbstractInlineLeafIO
+                || io instanceof InlineLeafIO;
+
+            return new BPlusLeafIoDelegate((BPlusLeafIO<IndexRow>)io, rowHnd);
+        }
+    }
+
+    /** */
+    private static <T extends BPlusIO<IndexRow> & InlineIO> IndexRow lookupRow(
+        InlineIndexRowHandler rowHnd,
+        long pageAddr,
+        int idx,
+        T io
+    ) {
+        long link = io.link(pageAddr, idx);
+
+        int off = io.offset(idx);
+
+        IndexKey[] keys = new IndexKey[rowHnd.indexKeyDefinitions().size()];
+
+        int fieldOff = 0;
+
+        for (int i = 0; i < rowHnd.inlineIndexKeyTypes().size(); i++) {
+            InlineIndexKeyType keyType = rowHnd.inlineIndexKeyTypes().get(i);
+
+            IndexKey key = keyType.get(pageAddr, off + fieldOff, io.inlineSize() - fieldOff);
+
+            fieldOff += keyType.inlineSize(key);
+
+            keys[i] = key;
+        }
+
+        return new IndexRowImpl(rowHnd, new CacheDataRowAdapter(link), keys);
+    }
+
+    /** */
+    private static class BPlusInnerIoDelegate<IO extends BPlusInnerIO<IndexRow> & InlineIO>
+        extends BPlusInnerIO<IndexRow> implements InlineIO {
+        /** */
+        private final IO io;
+
+        /** */
+        private final InlineIndexRowHandler rowHnd;
+
+        /** */
+        public BPlusInnerIoDelegate(IO io, InlineIndexRowHandler rowHnd) {

Review comment:
       fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/defragmentation/DefragIndexFactory.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.cache.query.index.sorted.defragmentation;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandlerFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.MetaPageInfo;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineRecommender;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.RootPage;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+
+/**
+ * Creates temporary index to defragment old index.
+ */
+public class DefragIndexFactory extends InlineIndexFactory {
+    /** Temporary offheap manager. */
+    private final IgniteCacheOffheapManager offheap;
+
+    /** Old index. */
+    private final InlineIndex oldIdx;
+
+    /** Temporary cache page memory. */
+    private final PageMemory newCachePageMemory;
+
+    /** */
+    private final InlineIndexRowHandlerFactory rowHndFactory;
+
+    /** */
+    public DefragIndexFactory(IgniteCacheOffheapManager offheap, PageMemory newCachePageMemory, InlineIndex oldIdx) {
+        // Row handler factory that produces no-op handler.
+        rowHndFactory = (def, settings) -> oldIdx.segment(0).rowHandler();
+
+        this.offheap = offheap;
+        this.oldIdx = oldIdx;
+        this.newCachePageMemory = newCachePageMemory;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected InlineIndexTree createIndexSegment(GridCacheContext<?, ?> cctx, SortedIndexDefinition def,
+        RootPage rootPage, IoStatisticsHolder stats, InlineRecommender recommender, int segmentNum) throws Exception {
+
+        InlineIndexTree tree = new InlineIndexTree(
+            def,
+            cctx,
+            def.treeName(),
+            offheap,
+            offheap.reuseListForIndex(def.treeName()),
+            newCachePageMemory,
+            // Use old row handler to have access to inline index key types.
+            pageIoResolver(),
+            rootPage.pageId().pageId(),
+            rootPage.isAllocated(),
+            oldIdx.inlineSize(),
+            def.keyTypeSettings(),
+            null,
+            stats,
+            rowHndFactory,
+            null
+        );
+
+        final MetaPageInfo oldInfo = oldIdx.segment(segmentNum).metaInfo();
+
+        tree.copyMetaInfo(oldInfo);
+
+        tree.enableSequentialWriteMode();
+
+        return tree;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RootPage rootPage(GridCacheContext<?, ?> ctx, String treeName, int segment) throws Exception {
+        return offheap.rootPageForIndex(ctx.cacheId(), treeName, segment);
+    }
+
+    /** */
+    private PageIoResolver pageIoResolver() {
+        return pageAddr -> {
+            PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+            if (io instanceof BPlusMetaIO)
+                return io;
+
+            //noinspection unchecked,rawtypes,rawtypes
+            return wrap((BPlusIO)io, rowHndFactory.create(null, null));
+        };
+    }
+
+    /** */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    static BPlusIO<IndexRow> wrap(BPlusIO<IndexRow> io, InlineIndexRowHandler rowHnd) {
+        assert io instanceof InlineIO;
+
+        if (io instanceof BPlusInnerIO) {
+            assert io instanceof AbstractInlineInnerIO
+                || io instanceof InlineInnerIO;
+
+            return new BPlusInnerIoDelegate((BPlusInnerIO<IndexRow>)io, rowHnd);
+        }
+        else {
+            assert io instanceof AbstractInlineLeafIO
+                || io instanceof InlineLeafIO;
+
+            return new BPlusLeafIoDelegate((BPlusLeafIO<IndexRow>)io, rowHnd);
+        }
+    }
+
+    /** */
+    private static <T extends BPlusIO<IndexRow> & InlineIO> IndexRow lookupRow(
+        InlineIndexRowHandler rowHnd,
+        long pageAddr,
+        int idx,
+        T io
+    ) {
+        long link = io.link(pageAddr, idx);
+
+        int off = io.offset(idx);
+
+        IndexKey[] keys = new IndexKey[rowHnd.indexKeyDefinitions().size()];
+
+        int fieldOff = 0;
+
+        for (int i = 0; i < rowHnd.inlineIndexKeyTypes().size(); i++) {
+            InlineIndexKeyType keyType = rowHnd.inlineIndexKeyTypes().get(i);
+
+            IndexKey key = keyType.get(pageAddr, off + fieldOff, io.inlineSize() - fieldOff);
+
+            fieldOff += keyType.inlineSize(key);
+
+            keys[i] = key;
+        }
+
+        return new IndexRowImpl(rowHnd, new CacheDataRowAdapter(link), keys);
+    }
+
+    /** */
+    private static class BPlusInnerIoDelegate<IO extends BPlusInnerIO<IndexRow> & InlineIO>
+        extends BPlusInnerIO<IndexRow> implements InlineIO {
+        /** */
+        private final IO io;
+
+        /** */
+        private final InlineIndexRowHandler rowHnd;
+
+        /** */
+        public BPlusInnerIoDelegate(IO io, InlineIndexRowHandler rowHnd) {

Review comment:
       fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractLeafIO.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.cache.query.index.sorted.inline.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.ThreadLocalRowHandlerHolder;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+
+/**
+ * Leaf page to store index rows.
+ */
+public abstract class AbstractLeafIO extends BPlusLeafIO<IndexRow> implements InlineIO {
+    /**
+     * @param type Page type.
+     * @param ver Page format version.
+     * @param itemSize Single item size on page.
+     */
+    AbstractLeafIO(int type, int ver, int itemSize) {
+        super(type, ver, itemSize);
+    }
+

Review comment:
       fixed.




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



[GitHub] [ignite] timoninmaxim edited a comment on pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim edited a comment on pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#issuecomment-775201344


   Hi @korlov42 ! Thanks a lot for reviewing my PR! You ask right questions. I worked on this PR pretty much time and then need to restore why I did some things this way. So I'm going to answer for one thing at a time.
   
   > dropping supporting of fixed-size and case insensitive strings
   
   Those 2 types actually aren't used in Ignite. 
   1. There is a ticket for supporting case insensitive things [IGNITE-3999](https://issues.apache.org/jira/browse/IGNITE-3999). Also some comments in this ticket suggest to implement it in different way, with introducing functional indexes.
   2. For fixed size strings. We work with them incorrectly and it looks like a bug. H2 provide a mapping: ValueStringFixed is used for char(), nchar(), character() types. But Ignite maps char() to String.class, then with H2Utils maps String.class to varchar, see [H2Utils.dbTypeFromClass](https://github.com/apache/ignite/blob/86073947248f0ca878e754e8b1b6181fdac72bd0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java#L709)
   
   I didn't have a problem to implement them, there will be just 2 classes that extends StringInlineIndexKeyType. But we can't test them now, as they will be not in use. Also it breaks compatibility for fixed strings (as currently it used type ValueString type 13, instead of 21 for ValueStringFixed).
   
   > closes doors for future improvement like [this one](https://issues.apache.org/jira/browse/IGNITE-13364).
   
   For task IGNITE-13364, I check a PR [PR/8161](https://github.com/apache/ignite/pull/8161/files) for this task and found that there is no problems to implement this feature within current changes. Actually there is `precision` field (GridH2RowDescriptor -> GridQueryProperty), that can be declared within IndexKeyDefinition, and used for computing inline size instead of parsing sql query.
   
   So, why do you think it will close door for new improvements?


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r575339674



##########
File path: modules/platforms/cpp/odbc-test/CMakeLists.txt
##########
@@ -62,7 +62,7 @@ set(SOURCES src/teamcity/teamcity_boost.cpp
         src/errors_test.cpp
         src/odbc_test_suite.cpp
         src/types_test.cpp
-        src/transaction_test.cpp
+#        src/transaction_test.cpp

Review comment:
       transaction_test.cpp uses TRANSACTIONAL_SNAPSHOT mode. As MVCC is deprecated I want to ignore this tests, but I've failed to find the right way to ignore them. So I comment them for a while. There are a lot of java tests that marked with Ignore now, I think can we just remove those tests at all? If yes I'd like to completely remove this CPP test too.




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



[GitHub] [ignite] timoninmaxim edited a comment on pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim edited a comment on pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#issuecomment-790029287


   Hi @korlov42 , @AMashenkov !
   
   Thanks for all comments. I've fixed most of them, the biggest changes are:
   1. Rework of hierarchies: IndexRow, IndexKey;
   2. Hide packages from public to internal;
   3. Restore backward compatibility for POJO type;
   4. Replace Schema with RowHandler;
   5. Introduce interfaces for date/time keys and delegates implementations to H2 (as it contains a lot of custom H2 logic, and license of h2 does not allow use this code, only in binary form).
   
   For most of other comments I provide fixes, comment them and resolve issues. Some of issues are not resolved to just clearify that my fix is correct.
   
   The only things to finalize:
   1. Cursor implementations can't be implemented other way as H2 ("select max(col) from table" query) depends on the logic that it invokes get() before next(). Currently I see a single way - put a comment there and fix with Calcite integration.
   2. MVCC tests, including C++ mvcc test, can I just remove all of them?
   3. If design is OK, then I will run performance tests.
   
   Could you please review my PR again?
   
   


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



[GitHub] [ignite] timoninmaxim commented on pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#issuecomment-775201344


   Hi @korlov42 ! Thanks a lot for reviewing my PR! You ask right questions. I worked on this PR pretty much time and then need to restore why I did some things this way. So I'm going to answer for one thing at a time.
   
   > dropping supporting of fixed-size and case insensitive strings
   
   Those 2 types actually aren't used in Ignite. 
   1. There is a ticket for supporting case insensitive things [IGNITE-3999](https://issues.apache.org/jira/browse/IGNITE-3999). Also some comments in this ticket suggest to implement it in different way, with introducing functional indexes.
   2. For fixed size strings. We work with them incorrectly and it looks like a bug. H2 provide a mapping: ValueStringFixed is used for char(), nchar(), character() types. But Ignite maps char() to String.class, then with H2Utils maps String.class to varchar, see [H2Utils.dbTypeFromClass](https://github.com/apache/ignite/blob/86073947248f0ca878e754e8b1b6181fdac72bd0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java#L709)
   
   I didn't have a problem to implement them, there will be just 2 classes that extends StringInlineIndexKeyType. But we can't test them now, as they will be not in use.
   
   > closes doors for future improvement like [this one](https://issues.apache.org/jira/browse/IGNITE-13364).
   
   For task IGNITE-13364, I check a PR (PR/8161)[https://github.com/apache/ignite/pull/8161/files] for this task and found that there is no problems to implement in with current changes. Actually there is `precision` field (GridH2RowDescriptor -> GridQueryProperty), that can be declared within IndexKeyDefinition, and used for computing inline size instead of parsing sql query.
   
   So, why do you think it will close door for new improvements?


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: [WIP] IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r548503107



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
##########
@@ -15,25 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;

Review comment:
       Actually I don't know why GitHub creates the link StringIgnoreCaseInlineIndexColumn -> AbstractIndex. `git log` for AbstractIndex is the only commit. I tried to git mv, git rm, create file from scratch again. But nothing helps and GitHub still show this again. 
   
   The same time Intellij IDEA shows history of this file differently, it saw it as GridQueryRowCacheCleaner -> AbstractIndex.
   
   Row git log is:
   
   ```
   % git log modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
   commit cee7855760b3fa1867a9e44e52be4ef786f108ad
   Author: Maksim Timonin <ti...@gmail.com>
   Date:   Thu Dec 10 01:25:18 2020 +0300
   
       Rebuild indexes, fix tests
   ```
   
   If somebody have any idea how to fix this, please share.
   




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r571627079



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)

Review comment:
       but now it extends a `GridCursor` which claims that `Initial state must be "before first"`. Current implementation violates our own contract




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r586747735



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/sorted/SortedIndex.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.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface for sorted Ignite indexes.
+ */
+public interface SortedIndex extends Index {
+    /**
+     * Finds index rows by specified range in specifed tree segment. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(@Nullable IndexKey lower, @Nullable IndexKey upper, int segment) throws IgniteCheckedException;
+
+    /**
+     * Finds index rows by specified range in specifed tree segment with cache filtering. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(IndexKey lower, IndexKey upper, int segment, IndexingQueryFilter filter)
+        throws IgniteCheckedException;
+
+    /**
+     * Finds first or last index row for specified tree segment and cache filter.
+     *
+     * @param firstOrLast if {@code true} then return first index row or otherwise last row.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> findFirstOrLast(boolean firstOrLast, int segment, IndexingQueryFilter filter)
+        throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment.
+     *
+     * @param segment Number of tree segment to find.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment with cache filter.
+     *
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment, IndexingQueryFilter filter) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows for all segments.
+     *
+     * @return total count of index rows.
+     */
+    public long totalCount() throws IgniteCheckedException;
+
+    /**
+     * Returns amount of index tree segments.
+     *
+     * @return amount of index tree segments.
+     */
+    public int segmentsCount();

Review comment:
       I think, that it's not worth to add complexity to this hierarchy, as currently there is no non-segmented indexes. I've just renamed SortedIndex to SortedSegmentedIndex.




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r575789779



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/IndexName.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.cache.query.index;
+
+import java.io.Serializable;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents list of names that fully describes index domain (schema, cache, table, index).
+ */
+public class IndexName implements Serializable {

Review comment:
       Hm, I forgot about PK. Ok, let's leave it as is for now




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r571664421



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
##########
@@ -464,41 +427,59 @@ public GridKernalContext kernalContext() {
      * @param unwrappedCols Unwrapped index columns for complex types.
      * @param wrappedCols Index columns as is complex types.
      * @param inlineSize Index inline size.
+     * @param cacheVisitor whether index created with new cache or on existing one.
      * @return Index.
      */
     @SuppressWarnings("ConstantConditions")
     GridH2IndexBase createSortedIndex(String name, GridH2Table tbl, boolean pk, boolean affinityKey,
-        List<IndexColumn> unwrappedCols, List<IndexColumn> wrappedCols, int inlineSize) {
-        try {
-            GridCacheContextInfo cacheInfo = tbl.cacheInfo();
-
-            if (log.isDebugEnabled())
-                log.debug("Creating cache index [cacheId=" + cacheInfo.cacheId() + ", idxName=" + name + ']');
-
-            if (cacheInfo.affinityNode()) {
-                final int segments = tbl.rowDescriptor().context().config().getQueryParallelism();
-
-                H2RowCache cache = rowCache.forGroup(cacheInfo.groupId());
-
-                return H2TreeIndex.createIndex(
-                    cacheInfo.cacheContext(),
-                    cache,
-                    tbl,
-                    name,
-                    pk,
-                    affinityKey,
-                    unwrappedCols,
-                    wrappedCols,
-                    inlineSize,
-                    segments,
-                    log
-                );
-            }
+        List<IndexColumn> unwrappedCols, List<IndexColumn> wrappedCols, int inlineSize, @Nullable SchemaIndexCacheVisitor cacheVisitor) {
+        GridCacheContextInfo cacheInfo = tbl.cacheInfo();
+
+        if (log.isDebugEnabled())
+            log.debug("Creating cache index [cacheId=" + cacheInfo.cacheId() + ", idxName=" + name + ']');
+
+        QueryIndexSchema schemaUnwrapped = new QueryIndexSchema(
+            tbl, unwrappedCols.toArray(new IndexColumn[0]));
+
+        QueryIndexSchema schemaWrapped = new QueryIndexSchema(
+            tbl, wrappedCols.toArray(new IndexColumn[0]));
+
+        if (cacheInfo.affinityNode()) {
+            SortedIndexDefinition idxDef = new QueryIndexDefinition(

Review comment:
       It's better to have an idxDefProvider at this point, because a final index definition will be available after metaPage is read (it depends on flags). Otherwise it bring a compatibility issues that will lead to an index tree corruption.
   
   Here are some rules to preserve the compatibility:
   - if `inlineObjects` flag is set to `false` then columns with type `JAVA_OBJECT` should be omitted. For example index from `(int, JO, int)` becomes `(int, int)`
   - if there is a type that currently unsupported, index's tail should be truncated up to this column. For example for index `(int, int, decimal, int, int)` only first two columns should be preserved.
   - if inlining of object's hash is not supported then `JAVA_OBJECT`'s should be treated as array of bytes

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IoStatisticsHolder stats,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        if (!initNew) {
+            // Init from metastore
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = getMetaInfo();
+
+            this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+            inlineSize = metaInfo.inlineSize();
+
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            this.def.setUseUnwrappedPk(true);
+
+            inlineSize = computeInlineSize(
+                def.getSchema().getKeyDefinitions(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+        }
+
+        if (inlineSize == 0)
+            setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+        else
+            setIos(
+                // -1 is required as payload starts with 1, and indexes in list of IOs are with 0.
+                (IOVersions<BPlusInnerIO<IndexRow>>) PageIO.getInnerVersions(inlineSize - 1, false),
+                (IOVersions<BPlusLeafIO<IndexRow>>) PageIO.getLeafVersions(inlineSize - 1, false));
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+        IndexSearchRow r = (IndexSearchRow) row;
+
+        int searchKeysLength = r.getSearchKeysCount();
+
+        if (inlineSize == 0)
+            return compareFullRows(getRow(io, pageAddr, idx), row, 0, searchKeysLength);
+
+        SortedIndexSchema schema = def.getSchema();
+
+        if ((schema.getKeyDefinitions().length != searchKeysLength) && r.isFullSchemaSearch())
+            throw new IgniteCheckedException("Find is configured for full schema search.");
+
+        int fieldOff = 0;
+
+        // Use it when can't compare values (variable length, for example).
+        int lastIdxUsed = searchKeysLength;
+
+        for (int i = 0; i < searchKeysLength; i++) {
+            try {
+                // If a search key is null then skip other keys (consider that null shows that we should get all
+                // possible keys for that comparison).
+                if (row.getKey(i) == null)
+                    return 0;
+
+                // Other keys are not inlined. Should compare as rows.
+                if (i >= schema.getKeyDefinitions().length) {
+                    lastIdxUsed = i;
+                    break;
+                }
+
+                int maxSize = inlineSize - fieldOff;
+
+                int off = io.offset(idx);
+
+                IndexKeyDefinition keyDef = schema.getKeyDefinitions()[i];
+
+                if (!InlineIndexKeyTypeRegistry.supportInline(keyDef.getIdxType())) {
+                    lastIdxUsed = i;
+                    break;
+                }
+
+                int cmp = COMPARE_UNSUPPORTED;
+
+                if (!row.getKey(i).getClass().isAssignableFrom(keyDef.getIdxClass())) {
+                    lastIdxUsed = i;
+                    break;
+                }
+
+                InlineIndexKeyType keyType = InlineIndexKeyTypeRegistry.get(keyDef.getIdxClass(), keyDef.getIdxType());
+
+                // By default do not compare different types.
+                if (InlineIndexKeyTypeRegistry.validate(keyDef.getIdxType(), row.getKey(i).getClass()))
+                    cmp = keyType.compare(pageAddr, off + fieldOff, maxSize, row.getKey(i));
+
+                // Can't compare as inlined bytes are not enough for comparation.
+                if (cmp == CANT_BE_COMPARE) {
+                    lastIdxUsed = i;
+                    break;
+                }
+
+                // Try compare stored values for inlined keys with different approach?
+                if (cmp == COMPARE_UNSUPPORTED)
+                    cmp = def.getRowComparator().compareKey(
+                        pageAddr, off + fieldOff, maxSize, row.getKey(i), keyType.type());
+
+                if (cmp == CANT_BE_COMPARE || cmp == COMPARE_UNSUPPORTED) {
+                    lastIdxUsed = i;
+                    break;
+                }
+
+                if (cmp != 0)
+                    return applySortOrder(cmp, schema.getKeyDefinitions()[i].getOrder().getSortOrder());
+
+                fieldOff += keyType.inlineSize(pageAddr, off + fieldOff);
+
+            } catch (Exception e) {
+                throw new IgniteException("Failed to store new index row.", e);
+            }
+        }
+
+        if (lastIdxUsed < searchKeysLength) {
+            recommender.recommend(row, inlineSize);
+
+            IndexRow currRow = getRow(io, pageAddr, idx);
+
+            for (int i = lastIdxUsed; i < searchKeysLength; i++) {
+                // If a search key is null then skip other keys (consider that null shows that we should get all
+                // possible keys for that comparison).
+                if (row.getKey(i) == null)
+                    return 0;
+
+                int c = def.getRowComparator().compareKey((IndexSearchRow) currRow, (IndexSearchRow) row, i);
+
+                if (c != 0)
+                    return applySortOrder(Integer.signum(c), schema.getKeyDefinitions()[i].getOrder().getSortOrder());
+            }

Review comment:
       ```suggestion
               return compareFullRows(currRow, row, lastIdxUsed, searchKeysLength)
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IoStatisticsHolder stats,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        if (!initNew) {
+            // Init from metastore
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = getMetaInfo();
+
+            this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+            inlineSize = metaInfo.inlineSize();
+
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            this.def.setUseUnwrappedPk(true);
+
+            inlineSize = computeInlineSize(
+                def.getSchema().getKeyDefinitions(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+        }
+
+        if (inlineSize == 0)
+            setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+        else
+            setIos(
+                // -1 is required as payload starts with 1, and indexes in list of IOs are with 0.
+                (IOVersions<BPlusInnerIO<IndexRow>>) PageIO.getInnerVersions(inlineSize - 1, false),
+                (IOVersions<BPlusLeafIO<IndexRow>>) PageIO.getLeafVersions(inlineSize - 1, false));
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+        IndexSearchRow r = (IndexSearchRow) row;
+
+        int searchKeysLength = r.getSearchKeysCount();
+
+        if (inlineSize == 0)
+            return compareFullRows(getRow(io, pageAddr, idx), row, 0, searchKeysLength);
+
+        SortedIndexSchema schema = def.getSchema();
+
+        if ((schema.getKeyDefinitions().length != searchKeysLength) && r.isFullSchemaSearch())
+            throw new IgniteCheckedException("Find is configured for full schema search.");
+
+        int fieldOff = 0;
+
+        // Use it when can't compare values (variable length, for example).
+        int lastIdxUsed = searchKeysLength;
+
+        for (int i = 0; i < searchKeysLength; i++) {
+            try {
+                // If a search key is null then skip other keys (consider that null shows that we should get all
+                // possible keys for that comparison).
+                if (row.getKey(i) == null)
+                    return 0;
+
+                // Other keys are not inlined. Should compare as rows.
+                if (i >= schema.getKeyDefinitions().length) {

Review comment:
       I'm afraid, this will lead to a performance degradation. For now `SortedIndexSchema#getKeyDefinitions()` return a copy of an array every time, but `compare` is on a hot path. Furthermore you are doing it several times for every search keys. It's better 1) to change key definitions to a `List`, 2) wrap it with `Collections#unmodifiableList` if you want to deny mutations, and 3) move it out from the loop.
   
   BTW, have you conducted any benchmarks to estimate an impact this refactoring have on a query performance?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IoStatisticsHolder stats,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        if (!initNew) {
+            // Init from metastore
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = getMetaInfo();
+
+            this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+            inlineSize = metaInfo.inlineSize();
+
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            this.def.setUseUnwrappedPk(true);
+
+            inlineSize = computeInlineSize(
+                def.getSchema().getKeyDefinitions(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+        }
+
+        if (inlineSize == 0)
+            setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+        else
+            setIos(
+                // -1 is required as payload starts with 1, and indexes in list of IOs are with 0.
+                (IOVersions<BPlusInnerIO<IndexRow>>) PageIO.getInnerVersions(inlineSize - 1, false),
+                (IOVersions<BPlusLeafIO<IndexRow>>) PageIO.getLeafVersions(inlineSize - 1, false));
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+        IndexSearchRow r = (IndexSearchRow) row;
+
+        int searchKeysLength = r.getSearchKeysCount();
+
+        if (inlineSize == 0)
+            return compareFullRows(getRow(io, pageAddr, idx), row, 0, searchKeysLength);
+
+        SortedIndexSchema schema = def.getSchema();
+
+        if ((schema.getKeyDefinitions().length != searchKeysLength) && r.isFullSchemaSearch())
+            throw new IgniteCheckedException("Find is configured for full schema search.");
+
+        int fieldOff = 0;
+
+        // Use it when can't compare values (variable length, for example).
+        int lastIdxUsed = searchKeysLength;
+
+        for (int i = 0; i < searchKeysLength; i++) {
+            try {
+                // If a search key is null then skip other keys (consider that null shows that we should get all
+                // possible keys for that comparison).
+                if (row.getKey(i) == null)
+                    return 0;
+
+                // Other keys are not inlined. Should compare as rows.
+                if (i >= schema.getKeyDefinitions().length) {
+                    lastIdxUsed = i;
+                    break;
+                }
+
+                int maxSize = inlineSize - fieldOff;
+
+                int off = io.offset(idx);

Review comment:
       looks like `off` is not changed through iteration too, so it's better to move it outside the loop

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IoStatisticsHolder stats,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        if (!initNew) {
+            // Init from metastore
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = getMetaInfo();
+
+            this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+            inlineSize = metaInfo.inlineSize();
+
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            this.def.setUseUnwrappedPk(true);
+
+            inlineSize = computeInlineSize(
+                def.getSchema().getKeyDefinitions(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+        }
+
+        if (inlineSize == 0)
+            setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+        else
+            setIos(
+                // -1 is required as payload starts with 1, and indexes in list of IOs are with 0.
+                (IOVersions<BPlusInnerIO<IndexRow>>) PageIO.getInnerVersions(inlineSize - 1, false),
+                (IOVersions<BPlusLeafIO<IndexRow>>) PageIO.getLeafVersions(inlineSize - 1, false));
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+        IndexSearchRow r = (IndexSearchRow) row;
+
+        int searchKeysLength = r.getSearchKeysCount();
+
+        if (inlineSize == 0)
+            return compareFullRows(getRow(io, pageAddr, idx), row, 0, searchKeysLength);
+
+        SortedIndexSchema schema = def.getSchema();
+
+        if ((schema.getKeyDefinitions().length != searchKeysLength) && r.isFullSchemaSearch())

Review comment:
       this should be verified only once before actual search. Also message error is not clear. Who does configure the search? What actually does "full schema search" means? I'd rather to get rid of this.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IoStatisticsHolder stats,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        if (!initNew) {
+            // Init from metastore
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = getMetaInfo();
+
+            this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+            inlineSize = metaInfo.inlineSize();
+
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            this.def.setUseUnwrappedPk(true);
+
+            inlineSize = computeInlineSize(
+                def.getSchema().getKeyDefinitions(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+        }
+
+        if (inlineSize == 0)
+            setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+        else
+            setIos(
+                // -1 is required as payload starts with 1, and indexes in list of IOs are with 0.
+                (IOVersions<BPlusInnerIO<IndexRow>>) PageIO.getInnerVersions(inlineSize - 1, false),
+                (IOVersions<BPlusLeafIO<IndexRow>>) PageIO.getLeafVersions(inlineSize - 1, false));
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+        IndexSearchRow r = (IndexSearchRow) row;
+
+        int searchKeysLength = r.getSearchKeysCount();
+
+        if (inlineSize == 0)
+            return compareFullRows(getRow(io, pageAddr, idx), row, 0, searchKeysLength);
+
+        SortedIndexSchema schema = def.getSchema();
+
+        if ((schema.getKeyDefinitions().length != searchKeysLength) && r.isFullSchemaSearch())
+            throw new IgniteCheckedException("Find is configured for full schema search.");
+
+        int fieldOff = 0;
+
+        // Use it when can't compare values (variable length, for example).
+        int lastIdxUsed = searchKeysLength;

Review comment:
       you could track it automatically by using instead of `I` in the loop




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r569555359



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/sorted/SortedIndex.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.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface for sorted Ignite indexes.
+ */
+public interface SortedIndex extends Index {
+    /**
+     * Finds index rows by specified range in specifed tree segment. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(@Nullable IndexKey lower, @Nullable IndexKey upper, int segment) throws IgniteCheckedException;
+
+    /**
+     * Finds index rows by specified range in specifed tree segment with cache filtering. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> find(IndexKey lower, IndexKey upper, int segment, IndexingQueryFilter filter)
+        throws IgniteCheckedException;
+
+    /**
+     * Finds first or last index row for specified tree segment and cache filter.
+     *
+     * @param firstOrLast if {@code true} then return first index row or otherwise last row.
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexSearchRow> findFirstOrLast(boolean firstOrLast, int segment, IndexingQueryFilter filter)
+        throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment.
+     *
+     * @param segment Number of tree segment to find.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment with cache filter.
+     *
+     * @param segment Number of tree segment to find.
+     * @param filter Cache entry filter.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment, IndexingQueryFilter filter) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows for all segments.
+     *
+     * @return total count of index rows.
+     */
+    public long totalCount() throws IgniteCheckedException;
+
+    /**
+     * Returns amount of index tree segments.
+     *
+     * @return amount of index tree segments.
+     */
+    public int segmentsCount();

Review comment:
       > actually segments has nothing common with a generic sorted index
   
   The `find` method already contains the segment argument in the signature. So `find` method of such general index will never be in use. For me when Ignite will need such general index it just extends existing one and hardcode segment with 0 value. WDYT?




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r586745142



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -2686,7 +2685,7 @@ private void finishUpdate(GridCacheContext cctx, CacheDataRow newRow, @Nullable
 
             GridCacheQueryManager qryMgr = cctx.queries();
 
-            if (qryMgr.enabled())
+            if (qryMgr.enabled() || cctx.kernalContext().indexing().enabled())

Review comment:
       Removed.




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



[GitHub] [ignite] asfgit closed pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #8490:
URL: https://github.com/apache/ignite/pull/8490


   


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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r571616976



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/Index.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for Ignite indexes.
+ */
+public interface Index {
+    /**
+     * Unique ID.
+     */
+    public UUID id();
+
+    /**
+     * Index name.
+     */
+    public String name();
+
+    /**
+     * Checks whether index handles specified cache row.
+     *
+     * @param row Cache row.
+     * @return Whether index handles specified cache row
+     */
+    public boolean belongsToIndex(CacheDataRow row) throws IgniteCheckedException;
+
+    /**
+     * Callback that runs when the underlying cache is updated.
+     *
+     * @param oldRow Cache row that was replaced with newRow.
+     * @param newRow Cache row that was stored.
+     * @param prevRowAvailable Whether oldRow available.
+     */
+    public void onUpdate(@Nullable CacheDataRow oldRow, @Nullable CacheDataRow newRow, boolean prevRowAvailable)
+        throws IgniteCheckedException;
+
+    /**
+     * Put cache row to index.
+     *
+     * @return {@code True} if replaced existing row.
+     */
+    public boolean putx(CacheDataRow row) throws IgniteCheckedException;

Review comment:
       well, I always interpret the `-x` suffix as `extended` (for example, with or without additional checks), and to have an extended version of something, you have to have the general one first. But it's just my intuition. And with these in mind I'm OK to leave this as is. Furthermore, since it will be moved to the internal space, we will be able to rename it late if needed.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r575308822



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IoStatisticsHolder stats,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        if (!initNew) {
+            // Init from metastore
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = getMetaInfo();
+
+            this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+            inlineSize = metaInfo.inlineSize();
+
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            this.def.setUseUnwrappedPk(true);
+
+            inlineSize = computeInlineSize(
+                def.getSchema().getKeyDefinitions(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+        }
+
+        if (inlineSize == 0)
+            setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+        else
+            setIos(
+                // -1 is required as payload starts with 1, and indexes in list of IOs are with 0.
+                (IOVersions<BPlusInnerIO<IndexRow>>) PageIO.getInnerVersions(inlineSize - 1, false),
+                (IOVersions<BPlusLeafIO<IndexRow>>) PageIO.getLeafVersions(inlineSize - 1, false));
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+        IndexSearchRow r = (IndexSearchRow) row;
+
+        int searchKeysLength = r.getSearchKeysCount();
+
+        if (inlineSize == 0)
+            return compareFullRows(getRow(io, pageAddr, idx), row, 0, searchKeysLength);
+
+        SortedIndexSchema schema = def.getSchema();
+
+        if ((schema.getKeyDefinitions().length != searchKeysLength) && r.isFullSchemaSearch())
+            throw new IgniteCheckedException("Find is configured for full schema search.");
+
+        int fieldOff = 0;
+
+        // Use it when can't compare values (variable length, for example).
+        int lastIdxUsed = searchKeysLength;
+
+        for (int i = 0; i < searchKeysLength; i++) {
+            try {
+                // If a search key is null then skip other keys (consider that null shows that we should get all
+                // possible keys for that comparison).
+                if (row.getKey(i) == null)
+                    return 0;
+
+                // Other keys are not inlined. Should compare as rows.
+                if (i >= schema.getKeyDefinitions().length) {

Review comment:
       Yes, the yardstick showed some degradation, but it wan't dramatically. I'd like to finalize that design is OK, then will fix issues with performance.
   
   > worth to cache a slice of array of key definitions
   
   Yes, I've implemented it. I will push the update as tests succeed. 
   




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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r601267704



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */

Review comment:
       typo 'wherther'




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r592134543



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get() {
+            return null;

Review comment:
       Create a ticket for that https://issues.apache.org/jira/browse/IGNITE-14303




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r571672872



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IoStatisticsHolder stats,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        if (!initNew) {
+            // Init from metastore
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = getMetaInfo();
+
+            this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+            inlineSize = metaInfo.inlineSize();
+
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            this.def.setUseUnwrappedPk(true);
+
+            inlineSize = computeInlineSize(
+                def.getSchema().getKeyDefinitions(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+        }
+
+        if (inlineSize == 0)
+            setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+        else
+            setIos(
+                // -1 is required as payload starts with 1, and indexes in list of IOs are with 0.
+                (IOVersions<BPlusInnerIO<IndexRow>>) PageIO.getInnerVersions(inlineSize - 1, false),
+                (IOVersions<BPlusLeafIO<IndexRow>>) PageIO.getLeafVersions(inlineSize - 1, false));
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+        IndexSearchRow r = (IndexSearchRow) row;
+
+        int searchKeysLength = r.getSearchKeysCount();
+
+        if (inlineSize == 0)
+            return compareFullRows(getRow(io, pageAddr, idx), row, 0, searchKeysLength);
+
+        SortedIndexSchema schema = def.getSchema();
+
+        if ((schema.getKeyDefinitions().length != searchKeysLength) && r.isFullSchemaSearch())

Review comment:
       this should be verified only once before actual search. Also message error is not clear. Who does configure the search? What actually does "full schema search" means? I'd rather get rid of this.




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



[GitHub] [ignite] timoninmaxim commented on pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#issuecomment-801145914


   Hi @korlov42 ! Thanks for the review. I've fixed all your comments. Some details:
   1. IndexName now is not Serializable. So I think there won't be any compatibility issues.
   2. IgniteIndexing now is IndexProcessor. I reverted all changes in IndexSpi.
   3. I added SignedBytesIndexKey to simplify the signature of the IndexKey.compare method.
   4. I reviewed all deleted or ignored tests and found 2 that was ignored incorrectly (they test clean index tree task). Restored them.
   5. The only entrypoint for index rows comparison is the `InlineIndexTree.compare` method. So I think it's OK that parts of the comparison mechanism are splitted between different classes, as all of them are used in single place.
   6. Performance test is OK, no degradation for sql queries.


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r575306523



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
##########
@@ -464,41 +427,59 @@ public GridKernalContext kernalContext() {
      * @param unwrappedCols Unwrapped index columns for complex types.
      * @param wrappedCols Index columns as is complex types.
      * @param inlineSize Index inline size.
+     * @param cacheVisitor whether index created with new cache or on existing one.
      * @return Index.
      */
     @SuppressWarnings("ConstantConditions")
     GridH2IndexBase createSortedIndex(String name, GridH2Table tbl, boolean pk, boolean affinityKey,
-        List<IndexColumn> unwrappedCols, List<IndexColumn> wrappedCols, int inlineSize) {
-        try {
-            GridCacheContextInfo cacheInfo = tbl.cacheInfo();
-
-            if (log.isDebugEnabled())
-                log.debug("Creating cache index [cacheId=" + cacheInfo.cacheId() + ", idxName=" + name + ']');
-
-            if (cacheInfo.affinityNode()) {
-                final int segments = tbl.rowDescriptor().context().config().getQueryParallelism();
-
-                H2RowCache cache = rowCache.forGroup(cacheInfo.groupId());
-
-                return H2TreeIndex.createIndex(
-                    cacheInfo.cacheContext(),
-                    cache,
-                    tbl,
-                    name,
-                    pk,
-                    affinityKey,
-                    unwrappedCols,
-                    wrappedCols,
-                    inlineSize,
-                    segments,
-                    log
-                );
-            }
+        List<IndexColumn> unwrappedCols, List<IndexColumn> wrappedCols, int inlineSize, @Nullable SchemaIndexCacheVisitor cacheVisitor) {
+        GridCacheContextInfo cacheInfo = tbl.cacheInfo();
+
+        if (log.isDebugEnabled())
+            log.debug("Creating cache index [cacheId=" + cacheInfo.cacheId() + ", idxName=" + name + ']');
+
+        QueryIndexSchema schemaUnwrapped = new QueryIndexSchema(
+            tbl, unwrappedCols.toArray(new IndexColumn[0]));
+
+        QueryIndexSchema schemaWrapped = new QueryIndexSchema(
+            tbl, wrappedCols.toArray(new IndexColumn[0]));
+
+        if (cacheInfo.affinityNode()) {
+            SortedIndexDefinition idxDef = new QueryIndexDefinition(

Review comment:
       Hi @korlov42 ! Sorry for the time delay. I'm working on your comments. Most of the things look fine but there are some failed tests, I'm trying to fix them now. I hope, will push updated for all your comments today or on Monday.
   
   I've lost compatibility for POJOs, my bad! I've added tests to the ignite-compatibility module and reproduced it, fix it. About other types (fixed strings, case insensitive strings) I wrote a comment below the PR, please check it.
   
   I didn't implement it as `idxDefProvider`. As IndexDefinition should be instantiated before index creation, as IndexFactory accepts it as a parameter. So, I've moved the logic of finalizing key types to RowHandler. It replaced Schema as you suggest in the previous comment.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r568696920



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
##########
@@ -15,25 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.cache.query.index;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringIgnoreCase;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static java.lang.Boolean.FALSE;
+import static java.lang.Boolean.TRUE;
 
 /**
- * Inline index column implementation for inlining strings ignore case.
+ * Abstract class for all Index implementations.
  */
-public class StringIgnoreCaseInlineIndexColumn extends StringInlineIndexColumn {
+public abstract class AbstractIndex implements Index {

Review comment:
       Currently there are descenders: GeoSpatialIndexImpl, InlineIndexImpl.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r572034957



##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexForceRebuildTest.java
##########
@@ -558,11 +558,11 @@ private void removeLogListener(IgniteEx ignite, LogListener lsnr) {
     /**
      * Indexing that blocks index rebuild until status request is completed.
      */
-    private static class BlockingIndexing extends IgniteH2Indexing {
+    private static class BlockingIndexing extends IndexesRebuildTask {

Review comment:
       ```suggestion
       private static class BlockingIndexesRebuildTask extends IndexesRebuildTask {
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/Index.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for Ignite indexes.
+ */
+public interface Index {
+    /**
+     * Unique ID.
+     */
+    public UUID id();
+
+    /**
+     * Index name.
+     */
+    public String name();
+
+    /**
+     * Checks whether index handles specified cache row.
+     *
+     * @param row Cache row.
+     * @return Whether index handles specified cache row
+     */
+    public boolean handlesRow(CacheDataRow row) throws IgniteCheckedException;

Review comment:
       ```suggestion
       public boolean canHandle(CacheDataRow row) throws IgniteCheckedException;
   ```

##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexRebuildStatusTest.java
##########
@@ -240,18 +240,14 @@ private void checkResult(CommandHandler handler, UUID... nodeIds) {
     /**
      * Indexing that blocks index rebuild until status request is completed.
      */
-    private static class BlockingIndexing extends IgniteH2Indexing {
-        /** {@inheritDoc} */
-        @Override protected void rebuildIndexesFromHash0(
-            GridCacheContext cctx,
-            SchemaIndexCacheVisitorClosure clo,
-            GridFutureAdapter<Void> rebuildIdxFut)
-        {
+    private static class BlockingIndexing extends IndexesRebuildTask {

Review comment:
       ```suggestion
       private static class BlockingIndexesRebuildTask extends IndexesRebuildTask {
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyType.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for inlined index columns. It's not a generic to provide opportunity compare different types.
+ */
+public interface InlineIndexKeyType {
+    /**
+     * Returns type of inlined column.
+     *
+     * @return Integer code of the column's value type.
+     */
+    public int type();
+
+    /**
+     * Returns size of inlined key. It contains system fields too (type, length).
+     */
+    public int inlineSize();
+
+    /**
+     * Returns required inline size for specified key.

Review comment:
       Javadoc looks unclear.
   Is it suggested inline size for specified key?
   Does system fields are taken into account?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.SystemProperty;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRowImpl;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Write to a log recommendation for inline size.
+ */
+public class InlineRecommender {
+    /** @see #IGNITE_THROTTLE_INLINE_SIZE_CALCULATION */
+    public static final int DFLT_THROTTLE_INLINE_SIZE_CALCULATION = 1_000;
+
+    /** */
+    @SystemProperty(value = "How often real invocation of inline size calculation will be skipped.", type = Long.class,
+        defaults = "" + DFLT_THROTTLE_INLINE_SIZE_CALCULATION)
+    public static final String IGNITE_THROTTLE_INLINE_SIZE_CALCULATION = "IGNITE_THROTTLE_INLINE_SIZE_CALCULATION";
+
+    /** Counter of inline size calculation for throttling real invocations. */
+    private final ThreadLocal<Long> inlineSizeCalculationCntr = ThreadLocal.withInitial(() -> 0L);
+
+    /** How often real invocation of inline size calculation will be skipped. */
+    private final int inlineSizeThrottleThreshold =
+        IgniteSystemProperties.getInteger(IGNITE_THROTTLE_INLINE_SIZE_CALCULATION,
+            DFLT_THROTTLE_INLINE_SIZE_CALCULATION);
+
+    /** Keep max calculated inline size for current index. */
+    private final AtomicInteger maxCalculatedInlineSize = new AtomicInteger();
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Index definition. */
+    private final SortedIndexDefinition def;
+
+    /** Constructor. */
+    public InlineRecommender(GridCacheContext cctx, SortedIndexDefinition def) {
+        log = cctx.kernalContext().indexing().getLogger();
+        this.def = def;
+    }
+
+    /**
+     * Calculate aggregate inline size for given indexes and log recommendation in case calculated size more than
+     * current inline size.
+     */
+    @SuppressWarnings({"ConditionalBreakInInfiniteLoop", "IfMayBeConditional"})
+    public void recommend(IndexRow row, int currInlineSize) {
+        // Do the check only for put operations.
+        if (row instanceof IndexSearchRowImpl)
+            return;
+
+        Long invokeCnt = inlineSizeCalculationCntr.get();
+
+        inlineSizeCalculationCntr.set(++invokeCnt);
+
+        boolean throttle = invokeCnt % inlineSizeThrottleThreshold != 0;
+
+        if (throttle)
+            return;

Review comment:
       As I understand InlineRecommender is created on per-Index basis.
   If so, we will allocate thread-local slot in thead-local table for each of index and will never free if when index is dropped.
   Usually, it doesn't looks like a good idea to allocate thread-local for user threads (that are not ignite pool thread).
   
   May be we can use some timestamp based approach? E.g. every 5 minutes, a thread that successfully CAS timestamp will log the recommendation.
   Or may be try to CAS a shared AtomicLong field optimistically (with no loop) and skip this step if CAS failed due to high load?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
##########
@@ -0,0 +1,468 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.index.AbstractIndex;
+import org.apache.ignite.cache.query.index.Index;
+import org.apache.ignite.cache.query.index.SingleCursor;
+import org.apache.ignite.cache.query.index.sorted.IndexKey;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexValueCursor;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderIndex;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.metastorage.pendingtask.DurableBackgroundTask;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
+
+/**
+ * Sorted index implementation.
+ */
+public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
+    /** Unique ID. */
+    private final UUID id = UUID.randomUUID();
+
+    /** Segments. */
+    private final InlineIndexTree[] segments;
+
+    /** Index function. */
+    private final SortedIndexDefinition def;
+
+    /** Name of underlying tree name. */
+    private final String treeName;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** */
+    private final IoStatisticsHolderIndex stats;
+
+    /** Constructor. */
+    public InlineIndexImpl(GridCacheContext<?, ?> cctx, SortedIndexDefinition def, InlineIndexTree[] segments,
+        IoStatisticsHolderIndex stats) {
+        this.cctx = cctx;
+        this.segments = segments.clone();
+        this.def = def;
+        treeName = def.getTreeName();
+        this.stats = stats;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<IndexRow> find(IndexKey lower, IndexKey upper, int segment) throws IgniteCheckedException {
+        return find(lower, upper, segment, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<IndexRow> find(IndexKey lower, IndexKey upper, int segment, IndexingQueryFilter filter) throws IgniteCheckedException {
+        validateConditions(lower, upper);
+
+        InlineTreeFilterClosure closure = getFilterClosure(filter);
+
+        IndexSearchRow rlower = (IndexSearchRow) lower;
+        IndexSearchRow rupper = (IndexSearchRow) upper;
+
+        // If it is known that only one row will be returned an optimization is employed
+        if (isSingleRowLookup(rlower, rupper)) {
+            try {
+                ThreadLocalSchemaHolder.setSchema(def.getSchema());
+
+                IndexRowImpl row = segments[segment].findOne(rlower, closure, null);
+
+                if (row == null || isExpired(row))
+                    return IndexValueCursor.EMPTY;
+
+                return new SingleCursor<>(row);
+
+            } finally {
+                ThreadLocalSchemaHolder.cleanSchema();
+            }
+        }
+
+        try {
+            ThreadLocalSchemaHolder.setSchema(def.getSchema());
+
+            return segments[segment].find(rlower, rupper, closure, null);
+
+        } finally {
+            ThreadLocalSchemaHolder.cleanSchema();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long count(int segment) throws IgniteCheckedException {
+        return segments[segment].size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long count(int segment, IndexingQueryFilter filter) throws IgniteCheckedException {
+        return segments[segment].size(getFilterClosure(filter));
+    }
+
+    /**
+     * Returns number of elements in the tree by scanning pages of the bottom (leaf) level.
+     *
+     * @return Number of elements in the tree.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Override public long totalCount() throws IgniteCheckedException {
+        long ret = 0;
+
+        for (int i = 0; i < segmentsCount(); i++)
+            ret += segments[i].size();
+
+        return ret;
+    }
+
+    /** */
+    private boolean isSingleRowLookup(IndexSearchRow lower, IndexSearchRow upper) throws IgniteCheckedException {
+        return def.isPrimary() && lower != null && lower.isFullSchemaSearch() && checkRowsTheSame(lower, upper);
+    }
+
+    /**
+     * Checks both rows are the same. <p/>
+     * Primarly used to verify both search rows are the same and we can apply
+     * the single row lookup optimization.

Review comment:
       ```suggestion
        * Checks both rows are the same. 
        * <p/>
        * Primarily used to verify if the single-row-lookup optimization can be applied.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -2686,7 +2685,7 @@ private void finishUpdate(GridCacheContext cctx, CacheDataRow newRow, @Nullable
 
             GridCacheQueryManager qryMgr = cctx.queries();
 
-            if (qryMgr.enabled())
+            if (qryMgr.enabled() || cctx.kernalContext().indexing().enabled())

Review comment:
       Is it possible to have a single flag for this?
   Seems, it make no sense having Indexing enabled, but qryMgr disabled.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IoStatisticsHolder stats,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        if (!initNew) {
+            // Init from metastore
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = getMetaInfo();
+
+            this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+            inlineSize = metaInfo.inlineSize();
+
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            this.def.setUseUnwrappedPk(true);
+
+            inlineSize = computeInlineSize(
+                def.getSchema().getKeyDefinitions(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+        }
+
+        if (inlineSize == 0)
+            setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+        else
+            setIos(
+                // -1 is required as payload starts with 1, and indexes in list of IOs are with 0.
+                (IOVersions<BPlusInnerIO<IndexRow>>) PageIO.getInnerVersions(inlineSize - 1, false),
+                (IOVersions<BPlusLeafIO<IndexRow>>) PageIO.getLeafVersions(inlineSize - 1, false));
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+        IndexSearchRow r = (IndexSearchRow) row;
+
+        int searchKeysLength = r.getSearchKeysCount();
+
+        if (inlineSize == 0)
+            return compareFullRows(getRow(io, pageAddr, idx), row, 0, searchKeysLength);
+
+        SortedIndexSchema schema = def.getSchema();
+
+        if ((schema.getKeyDefinitions().length != searchKeysLength) && r.isFullSchemaSearch())
+            throw new IgniteCheckedException("Find is configured for full schema search.");
+
+        int fieldOff = 0;
+
+        // Use it when can't compare values (variable length, for example).
+        int lastIdxUsed = searchKeysLength;
+
+        for (int i = 0; i < searchKeysLength; i++) {
+            try {
+                // If a search key is null then skip other keys (consider that null shows that we should get all
+                // possible keys for that comparison).
+                if (row.getKey(i) == null)
+                    return 0;
+
+                // Other keys are not inlined. Should compare as rows.
+                if (i >= schema.getKeyDefinitions().length) {

Review comment:
       Also, there is a check few lines below
   " if (!InlineIndexKeyTypeRegistry.supportInline(keyDef.getIdxType())) break"
   I think, it worth to cache a slice of array of key definitions which support inlining.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
##########
@@ -15,25 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.cache.query.index;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringIgnoreCase;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static java.lang.Boolean.FALSE;
+import static java.lang.Boolean.TRUE;
 
 /**
- * Inline index column implementation for inlining strings ignore case.
+ * Abstract class for all Index implementations.
  */
-public class StringIgnoreCaseInlineIndexColumn extends StringInlineIndexColumn {
+public abstract class AbstractIndex implements Index {
+    /** Whether index is rebuilding now. */
+    private final AtomicBoolean rebuildInProgress = new AtomicBoolean(false);
+
     /**
-     * @param col Column.
+     * @param val Mark or unmark index to rebuild.
      */
-    public StringIgnoreCaseInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_IGNORECASE, useOptimizedCompare, true);
+    public void markIndexRebuild(boolean val) {
+        rebuildInProgress.compareAndSet(val ? FALSE : TRUE, val ? TRUE : FALSE);

Review comment:
       ```suggestion
           rebuildInProgress.compareAndSet(!val, val);
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyType.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for inlined index columns. It's not a generic to provide opportunity compare different types.
+ */
+public interface InlineIndexKeyType {
+    /**
+     * Returns type of inlined column.
+     *
+     * @return Integer code of the column's value type.
+     */
+    public int type();
+
+    /**
+     * Returns size of inlined key. It contains system fields too (type, length).

Review comment:
       ```suggestion
        * Returns size of inlined key. 
        *
        * Note: system fields (e.g. type, length) are taken into account as well.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/keys/StringInlineIndexKeyType.java
##########
@@ -15,62 +15,42 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.keys;
 
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexKeyTypes;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueString;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Inline index column implementation for inlining {@link String} values.
+ * Inline index key implementation for inlining {@link String} values.
  */
-public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
+public class StringInlineIndexKeyType extends NullableInlineIndexKeyType<String> {
     /** Default charset. */
     protected static final Charset CHARSET = StandardCharsets.UTF_8;
 
+    // TODO: how to configure it? Looks like ignoreCase may be configured on moment of Search row creation.

Review comment:
       Unrelated change.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/IndexKeyTypes.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ public static final int * = The ASF licenses this file to You under the Apache License, Version 2;
+ * (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
+ *
+ public static final int * = Unless required by applicable law or agreed to in writing, softwa;
+ public static final int * = distributed under the License is distributed on an "AS IS" BASI;
+ public static final int * = WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implie;

Review comment:
       Looks like unrelated change.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/ThreadLocalSchemaHolder.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.cache.query.index.sorted.inline.io;
+
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+
+/**
+ * Holds an index schema during work session with an index tree.
+ */
+public class ThreadLocalSchemaHolder {
+    /** */
+    private static final ThreadLocal<SortedIndexSchema> holder = new ThreadLocal<>();

Review comment:
       When you use ThreadLocal to pass some temporary context and call ThreadLocal.clear() on hot-path
   it worth to init ThreadLocal with a Holder object and update the Holder field directly rather then set-then-clear ThreadLocal itself, because ThreadLocal initialization requires 2 lookups.
   1-st lookup just to found it is not initialized (or with 'null' value), and the 2-nd to initialize.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/IndexKeyTypes.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ public static final int * = The ASF licenses this file to You under the Apache License, Version 2;

Review comment:
       Unrelated change.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/keys/StringInlineIndexKeyType.java
##########
@@ -15,62 +15,42 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.keys;
 
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexKeyTypes;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueString;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Inline index column implementation for inlining {@link String} values.
+ * Inline index key implementation for inlining {@link String} values.
  */
-public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
+public class StringInlineIndexKeyType extends NullableInlineIndexKeyType<String> {
     /** Default charset. */
     protected static final Charset CHARSET = StandardCharsets.UTF_8;
 
+    // TODO: how to configure it? Looks like ignoreCase may be configured on moment of Search row creation.
     /** Whether respect case or not while comparing. */
     private final boolean compareIgnoreCase;
 
-    /** Whether to use optimized comparison or not. */
-    private final boolean useOptimizedCompare;
+    // TODO: what is it?
+//    /** Whether to use optimized comparison or not. */
+//    private final boolean useOptimizedCompare;

Review comment:
       See h2.value.CompareMode. Flag useOptimizedCompare is false only if CompareMode.OFF is set.
   Seems, it is some kind of workaround when inlined Strings can't be compared directly.
   
   Let's either keep previous behavior of drop flag and remove TODO.

##########
File path: modules/platforms/cpp/odbc-test/CMakeLists.txt
##########
@@ -62,7 +62,7 @@ set(SOURCES src/teamcity/teamcity_boost.cpp
         src/errors_test.cpp
         src/odbc_test_suite.cpp
         src/types_test.cpp
-        src/transaction_test.cpp
+#        src/transaction_test.cpp

Review comment:
       Whats wrong with CPP tests?
   Is there any ticket for fixing them?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get() {
+            return null;

Review comment:
       Agree with @korlov42 , exception will be better here.
   As for now, this is contract voilation and cursor.get() may return 'null' in both cases 
   if there is a null-value and if there is no values.
   
   @timoninmaxim , let's fix this and run tests or create a separate ticket if there are any issues. WDYT?




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



[GitHub] [ignite] timoninmaxim edited a comment on pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim edited a comment on pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#issuecomment-775201344


   Hi @korlov42 ! Thanks a lot for reviewing my PR! You ask right questions. I worked on this PR pretty much time and then need to restore why I did some things this way. So I'm going to answer for one thing at a time.
   
   > dropping supporting of fixed-size and case insensitive strings
   
   Those 2 types actually aren't used in Ignite. 
   1. There is a ticket for supporting case insensitive things [IGNITE-3999](https://issues.apache.org/jira/browse/IGNITE-3999). Also some comments in this ticket suggest to implement it in different way, with introducing functional indexes.
   2. For fixed size strings. We work with them incorrectly and it looks like a bug. H2 provide a mapping: ValueStringFixed is used for char(), nchar(), character() types. But Ignite maps char() to String.class, then with H2Utils maps String.class to varchar, see [H2Utils.dbTypeFromClass](https://github.com/apache/ignite/blob/86073947248f0ca878e754e8b1b6181fdac72bd0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java#L709)
   
   I didn't have a problem to implement them, there will be just 2 classes that extends StringInlineIndexKeyType. But we can't test them now, as they will be not in use.
   
   > closes doors for future improvement like [this one](https://issues.apache.org/jira/browse/IGNITE-13364).
   
   For task IGNITE-13364, I check a PR [PR/8161](https://github.com/apache/ignite/pull/8161/files) for this task and found that there is no problems to implement in with current changes. Actually there is `precision` field (GridH2RowDescriptor -> GridQueryProperty), that can be declared within IndexKeyDefinition, and used for computing inline size instead of parsing sql query.
   
   So, why do you think it will close door for new improvements?


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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r571616708



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
##########
@@ -15,25 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.cache.query.index;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringIgnoreCase;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static java.lang.Boolean.FALSE;
+import static java.lang.Boolean.TRUE;
 
 /**
- * Inline index column implementation for inlining strings ignore case.
+ * Abstract class for all Index implementations.
  */
-public class StringIgnoreCaseInlineIndexColumn extends StringInlineIndexColumn {
+public abstract class AbstractIndex implements Index {

Review comment:
       It appears a 'geospatial' module was ignored in the project. Ok then




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r579449491



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyImpl.java
##########
@@ -15,41 +15,48 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
-import org.apache.ignite.internal.pagemem.PageUtils;
+import java.util.Arrays;
+import org.apache.ignite.cache.query.index.sorted.IndexKey;
 
 /**
- *
+ * Complex index key that represents a user index query.
  */
-class H2MvccExtrasInnerIO extends AbstractH2ExtrasInnerIO {
-    /**
-     * @param type Page type.
-     * @param ver Page format version.
-     * @param payloadSize Payload size.
-     */
-    H2MvccExtrasInnerIO(short type, int ver, int payloadSize) {
-        super(type, ver, 28, payloadSize);
+public class IndexKeyImpl implements IndexKey {

Review comment:
       Removed




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r568727179



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/IndexName.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.cache.query.index;
+
+import java.io.Serializable;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents list of names that fully describes index domain (schema, cache, table, index).
+ */
+public class IndexName implements Serializable {

Review comment:
       It's not actually correct for case of PR indexes. Index name for any PK will be "_key_PK". So for making purely unique index name we should add table name. 
   
   CacheName may be unnecessary, but I add it for non-sql indexes where schema name is null. 




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



[GitHub] [ignite] timoninmaxim commented on pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#issuecomment-790029287


   Hi @korlov42 , @AMashenkov !
   
   Thanks for all comments. I've fixed most of them, the biggest changes are:
   1. Rework of hierarchies: IndexRow, IndexKey;
   2. Hide packages from public to internal;
   3. Restore backward compatibility for POJO type;
   4. Replace Schema with RowHandler.
   5. Introduce interfaces for date/time keys and delegates implementations to H2 (as it contains a lot of custom H2 logic, and license of h2 does not allow use this code, only in binary form).
   
   For most of other comments I provide fixes, comment them and resolve issues. Some of issues are not resolved to just clearify that my fix is correct.
   
   The only things to finalize:
   1. Cursor implementations can't be implemented other way as H2 ("select max(col) from table" query) depends on the logic that it invokes get() before next(). Currently I see a single way - put a comment there and fix with Calcite integration. Or 
   2. MVCC tests, including C++ mvcc test, can I just remove all of them?
   3. If design is OK, then I will run performance tests.
   
   Could you please review my PR again?
   
   


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r579449961



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/DefaultIndexRowComparator.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * Default row comparator. Consider that every index key extends Comparable interface.
+ * Does not support comparation of different key types.
+ */
+public class DefaultIndexRowComparator implements IndexRowComparator {

Review comment:
       Removed




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r598562780



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/InlineIndexColumnTest.java
##########
@@ -1,1014 +0,0 @@
-/*
- * 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.h2.database.inlinecolumn;
-
-import java.io.Serializable;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.TimeZone;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-import org.apache.commons.io.Charsets;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.configuration.DataRegionConfiguration;
-import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
-import org.apache.ignite.internal.pagemem.PageIdAllocator;
-import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
-import org.apache.ignite.internal.processors.cache.index.AbstractIndexingCommonTest;
-import org.apache.ignite.internal.processors.metric.impl.LongAdderMetric;
-import org.apache.ignite.internal.processors.query.h2.database.InlineIndexColumn;
-import org.apache.ignite.testframework.junits.GridTestBinaryMarshaller;
-import org.apache.ignite.testframework.junits.WithSystemProperty;
-import org.h2.table.Column;
-import org.h2.util.DateTimeUtils;
-import org.h2.value.CompareMode;
-import org.h2.value.Value;
-import org.h2.value.ValueBoolean;
-import org.h2.value.ValueByte;
-import org.h2.value.ValueBytes;
-import org.h2.value.ValueDate;
-import org.h2.value.ValueDouble;
-import org.h2.value.ValueFloat;
-import org.h2.value.ValueInt;
-import org.h2.value.ValueJavaObject;
-import org.h2.value.ValueLong;
-import org.h2.value.ValueNull;
-import org.h2.value.ValueShort;
-import org.h2.value.ValueString;
-import org.h2.value.ValueStringFixed;
-import org.h2.value.ValueStringIgnoreCase;
-import org.h2.value.ValueTime;
-import org.h2.value.ValueTimestamp;
-import org.h2.value.ValueUuid;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.apache.ignite.internal.processors.query.h2.database.inlinecolumn.AbstractInlineIndexColumn.CANT_BE_COMPARE;
-
-/**
- * Simple tests for {@link InlineIndexColumn}.
- */
-@WithSystemProperty(key = "h2.serializeJavaObject", value = "false")
-@WithSystemProperty(key = "h2.objectCache", value = "false")
-public class InlineIndexColumnTest extends AbstractIndexingCommonTest {

Review comment:
       Do we have replacement for this test?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java
##########
@@ -1,163 +0,0 @@
-/*
- * 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.h2;
-
-import java.util.Iterator;
-import java.util.Map;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.PageIdUtils;
-import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.processors.cache.CacheGroupContext;
-import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
-import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
-import org.apache.ignite.internal.util.typedef.F;
-import org.jsr166.ConcurrentLinkedHashMap;
-
-import static org.jsr166.ConcurrentLinkedHashMap.DFLT_INIT_CAP;
-import static org.jsr166.ConcurrentLinkedHashMap.DFLT_LOAD_FACTOR;
-
-/**
- * H2 row cache.
- */
-public class H2RowCache implements GridQueryRowCacheCleaner {

Review comment:
       I don't sure it's a good idea to remove row cache under within this PR. If the row cache is really useless, it's better to start a discussion on the dev list first.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r568696920



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/AbstractIndex.java
##########
@@ -15,25 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.cache.query.index;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringIgnoreCase;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static java.lang.Boolean.FALSE;
+import static java.lang.Boolean.TRUE;
 
 /**
- * Inline index column implementation for inlining strings ignore case.
+ * Abstract class for all Index implementations.
  */
-public class StringIgnoreCaseInlineIndexColumn extends StringInlineIndexColumn {
+public abstract class AbstractIndex implements Index {

Review comment:
       Currently there are descenders: GeoSpatialIndexImpl, InlineIndexImpl.

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/Index.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for Ignite indexes.
+ */
+public interface Index {
+    /**
+     * Unique ID.
+     */
+    public UUID id();
+
+    /**
+     * Index name.
+     */
+    public String name();
+
+    /**
+     * Checks whether index handles specified cache row.
+     *
+     * @param row Cache row.
+     * @return Whether index handles specified cache row
+     */
+    public boolean belongsToIndex(CacheDataRow row) throws IgniteCheckedException;
+
+    /**
+     * Callback that runs when the underlying cache is updated.
+     *
+     * @param oldRow Cache row that was replaced with newRow.
+     * @param newRow Cache row that was stored.
+     * @param prevRowAvailable Whether oldRow available.
+     */
+    public void onUpdate(@Nullable CacheDataRow oldRow, @Nullable CacheDataRow newRow, boolean prevRowAvailable)
+        throws IgniteCheckedException;
+
+    /**
+     * Put cache row to index.
+     *
+     * @return {@code True} if replaced existing row.
+     */
+    public boolean putx(CacheDataRow row) throws IgniteCheckedException;

Review comment:
       The previous implementation of H2TreeIndex used the `putx` method only. It did have the `put` method, but in all places it's not used returned value, just checks it for existing, so it's similar to `putx` logic. So it looks like it's just not required. I can add it to make API more wide, but as it won't be a part of public for a while API, do we need it?
   
   

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/IndexName.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.cache.query.index;
+
+import java.io.Serializable;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents list of names that fully describes index domain (schema, cache, table, index).
+ */
+public class IndexName implements Serializable {

Review comment:
       It's not actually correct for case of PR indexes. Index name for any PK will be "_key_PK". So for making purely unique index name we should add table name. 
   
   CacheName may be unnecessary, but I add it for non-sql indexes where schema name is null. 

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/IndexName.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.cache.query.index;
+
+import java.io.Serializable;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents list of names that fully describes index domain (schema, cache, table, index).
+ */
+public class IndexName implements Serializable {

Review comment:
       It's not actually correct for case of PK indexes. Index name for any PK will be "_key_PK". So for making purely unique index name we should add table name. 
   
   CacheName may be unnecessary, but I add it for non-sql indexes where schema name is null. 

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)

Review comment:
       Yes, it looks pretty strange. But it's the same logic as org.h2.index.SingleRowCursor does ([Link](https://github.com/h2database/h2database/blob/version-1.4.197/h2/src/main/org/h2/index/SingleRowCursor.java))
   
   Also we have some tests (or logic) that differently works with SingleCursor. Some of them just use get, some of them make `next -> get` 

##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)

Review comment:
       Yes, it looks pretty strange. But it's the same logic as org.h2.index.SingleRowCursor does ([Link](https://github.com/h2database/h2database/blob/version-1.4.197/h2/src/main/org/h2/index/SingleRowCursor.java))
   
   Also we have some tests (or logic) that differently works with SingleCursor. Some of them just use `get`, some of them make `next -> get` 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get() {
+            return null;

Review comment:
       It's implemented the way it was before. Please check GridH2Cursor.EMPTY [Link](https://github.com/apache/ignite/blob/2d2044a63440e9b9d8495f35819ccee26d8324e4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Cursor.java#L31)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get() {
+            return null;

Review comment:
       It's implemented the way it was before. Please check GridH2Cursor.EMPTY ([Link](https://github.com/apache/ignite/blob/2d2044a63440e9b9d8495f35819ccee26d8324e4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Cursor.java#L31))




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r569520969



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexSchema.java
##########
@@ -15,25 +15,40 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringFixed;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 
 /**
- * Inline index column implementation for inlining strings of fixed length.
+ * Schema for sorted index.
  */
-public class FixedStringInlineIndexColumn extends StringInlineIndexColumn {
+public interface SortedIndexSchema {
     /**
-     * @param col Column.
+     * Describe all index keys.
      */
-    public FixedStringInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_FIXED, useOptimizedCompare, false);
-    }
+    public IndexKeyDefinition[] getKeyDefinitions();

Review comment:
       Schema and Definition are for different purposes. 
   
   - IndexDefinition contains instructions on how to create an index and which parameters it has.
   - Schema is responsible for mapping cache entry on index rows. 
   
   So schema isn't just a list of keys. For example, `QueryIndexSchema` relies on the `GridH2RowDescriptor` object to provide the mapping. IOs and IndexRow require this mapping, but they actually should not have access to IndexDefinition.
   
   Maybe just "Schema" is not a correct name for this target?
   
   




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r594593169



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.cache.query.index.sorted.keys;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+
+/** */
+public class BooleanIndexKey implements IndexKey {
+    /** */
+    private final boolean key;
+
+    /** */
+    public BooleanIndexKey(boolean key) {
+        this.key = key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getKey() {
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getType() {
+        return IndexKeyTypes.BOOLEAN;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(IndexKey o, IndexKeyTypeSettings keySettings) {

Review comment:
       Provided a SignedBytesIndexKey




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r592134438



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)

Review comment:
       Create a ticket for that. https://issues.apache.org/jira/browse/IGNITE-14303




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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r601264893



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexFactory.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.cache.query.index;
+
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Base interface for Ignite index factories.
+ */
+public interface IndexFactory {
+    /**
+     * Creates index by specifed defition for specified cache.

Review comment:
       typos




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r582084213



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
##########
@@ -0,0 +1,480 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.index.sorted.SortOrder;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexSchema;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.ThreadLocalSchemaHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.keys.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IoStatisticsHolder stats,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        if (!initNew) {
+            // Init from metastore
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = getMetaInfo();
+
+            this.def.setUseUnwrappedPk(metaInfo.useUnwrappedPk());
+
+            inlineSize = metaInfo.inlineSize();
+
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            this.def.setUseUnwrappedPk(true);
+
+            inlineSize = computeInlineSize(
+                def.getSchema().getKeyDefinitions(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+        }
+
+        if (inlineSize == 0)
+            setIos(InnerIO.VERSIONS, LeafIO.VERSIONS);
+        else
+            setIos(
+                // -1 is required as payload starts with 1, and indexes in list of IOs are with 0.
+                (IOVersions<BPlusInnerIO<IndexRow>>) PageIO.getInnerVersions(inlineSize - 1, false),
+                (IOVersions<BPlusLeafIO<IndexRow>>) PageIO.getLeafVersions(inlineSize - 1, false));
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+
+        this.cctx = cctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+        IndexSearchRow r = (IndexSearchRow) row;
+
+        int searchKeysLength = r.getSearchKeysCount();
+
+        if (inlineSize == 0)
+            return compareFullRows(getRow(io, pageAddr, idx), row, 0, searchKeysLength);
+
+        SortedIndexSchema schema = def.getSchema();
+
+        if ((schema.getKeyDefinitions().length != searchKeysLength) && r.isFullSchemaSearch())
+            throw new IgniteCheckedException("Find is configured for full schema search.");
+
+        int fieldOff = 0;
+
+        // Use it when can't compare values (variable length, for example).
+        int lastIdxUsed = searchKeysLength;
+
+        for (int i = 0; i < searchKeysLength; i++) {
+            try {
+                // If a search key is null then skip other keys (consider that null shows that we should get all
+                // possible keys for that comparison).
+                if (row.getKey(i) == null)
+                    return 0;
+
+                // Other keys are not inlined. Should compare as rows.
+                if (i >= schema.getKeyDefinitions().length) {

Review comment:
       Replaced array with unmodified collection. 




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r592226761



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexSearchRow.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+
+/**
+ * Represents a complex index key.
+ */
+public interface IndexSearchRow {

Review comment:
       Agree, for internal usage it is OK to merge them. The method `getKeys()` is used for defragmentation to skip extract values from an ignite cache twice.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r582067210



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/keys/StringInlineIndexKeyType.java
##########
@@ -15,62 +15,42 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.keys;
 
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexKeyTypes;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueString;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Inline index column implementation for inlining {@link String} values.
+ * Inline index key implementation for inlining {@link String} values.
  */
-public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
+public class StringInlineIndexKeyType extends NullableInlineIndexKeyType<String> {
     /** Default charset. */
     protected static final Charset CHARSET = StandardCharsets.UTF_8;
 
+    // TODO: how to configure it? Looks like ignoreCase may be configured on moment of Search row creation.
     /** Whether respect case or not while comparing. */
     private final boolean compareIgnoreCase;
 
-    /** Whether to use optimized comparison or not. */
-    private final boolean useOptimizedCompare;
+    // TODO: what is it?
+//    /** Whether to use optimized comparison or not. */
+//    private final boolean useOptimizedCompare;

Review comment:
       Restored previous behavior.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r569490636



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/sorted/IndexKey.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents a complex index key.
+ */
+public interface IndexKey {

Review comment:
       Yes, I agree, this is wrong. As IndexKey expected to be used by user queries and it definitely should not contain info about CacheDataRow.
   
   I've reworked a hierarchy of classes related to IndexKey, IndexRow. Now InlineIndexTree is generic of IndexRow, and not of IndexSearchRow (it was confusing, IMHO). IndexRow now extends IndexKey. As a result, I simplified a couple of places in the code.




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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r601252085



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedSegmentedIndex.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface for sorted and segmented Ignite indexes.
+ */
+public interface SortedSegmentedIndex extends Index {
+    /**
+     * Finds index rows by specified range in specifed tree segment. Range can be bound or unbound.

Review comment:
       typo: 'specifed' -> 'specified' and the same typo below




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r592499886



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/DurableBackgroundCleanupIndexTreeTask.java
##########
@@ -46,53 +49,44 @@
     private List<Long> rootPages;
 
     /** */
-    private transient List<H2Tree> trees;
+    private transient List<InlineIndexTree> trees;
 
     /** */
     private transient volatile boolean completed;
 
     /** */
-    private String cacheGrpName;
+    private final String cacheName;
 
     /** */
-    private String cacheName;
+    private final IndexName idxName;

Review comment:
       Hi, I will just remove Serializable. The reason to use it is a `DurableBackgroundCleanupIndexTreeTask`. It requires an index name for work. It was String, I replaced it with IndexName, then I will just revert this change.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r568727179



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/IndexName.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.cache.query.index;
+
+import java.io.Serializable;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents list of names that fully describes index domain (schema, cache, table, index).
+ */
+public class IndexName implements Serializable {

Review comment:
       It's not actually correct for case of PK indexes. Index name for any PK will be "_key_PK". So for making purely unique index name we should add table name. 
   
   CacheName may be unnecessary, but I add it for non-sql indexes where schema name is null. 




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r571652445



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexSchema.java
##########
@@ -15,25 +15,40 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringFixed;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 
 /**
- * Inline index column implementation for inlining strings of fixed length.
+ * Schema for sorted index.
  */
-public class FixedStringInlineIndexColumn extends StringInlineIndexColumn {
+public interface SortedIndexSchema {
     /**
-     * @param col Column.
+     * Describe all index keys.
      */
-    public FixedStringInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_FIXED, useOptimizedCompare, false);
-    }
+    public IndexKeyDefinition[] getKeyDefinitions();

Review comment:
       > IndexDefinition contains instructions on how to create an index and which parameters it has
   
   and with this in mind IndexDefinition is the best place for IndexKeyDefinition[]. Furthermore this is the way it's defined in IEP.
   
   BTW why have you decided not to follow abstractions defined in IEP?
   
   > Schema is responsible for mapping cache entry on index rows.
   
   It's better to introduce something like this: 
   ```
   public interface RowHandler<Row> {
       /** */
       Object get(int field, Row row);
   
       /** */
       int columnCount(Row row);
   }
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r575337978



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -2686,7 +2685,7 @@ private void finishUpdate(GridCacheContext cctx, CacheDataRow newRow, @Nullable
 
             GridCacheQueryManager qryMgr = cctx.queries();
 
-            if (qryMgr.enabled())
+            if (qryMgr.enabled() || cctx.kernalContext().indexing().enabled())

Review comment:
       > no sense having Indexing enabled, but qryMgr disabled.
   
   Hi. It was a workaround, because for some reasons this condition was required at beginning of development. Currently I don't see in code any reasons to have this condition. So I'll remove it and check that everything is fine.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r568740505



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)

Review comment:
       Yes, it looks pretty strange. But it's the same logic as org.h2.index.SingleRowCursor does ([Link](https://github.com/h2database/h2database/blob/version-1.4.197/h2/src/main/org/h2/index/SingleRowCursor.java))
   
   Also we have some tests (or logic) that differently works with SingleCursor. Some of them just use get, some of them make `next -> get` 




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r594554427



##########
File path: modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
##########
@@ -84,7 +91,56 @@
      * @param expirationTime Expiration time or 0 if never expires.
      * @throws IgniteSpiException If failed.
      */
-    public void store(@Nullable String cacheName, Object key, Object val, long expirationTime) throws IgniteSpiException;
+    public default void store(@Nullable String cacheName, Object key, Object val, long expirationTime) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /**
+     * Updates index with new row. Note that key is unique for cache, so if cache contains multiple indexes
+     * the key should be removed from indexes other than one being updated.
+     *
+     * @param cctx Cache context.
+     * @param newRow cache row to store in index.
+     * @param prevRow optional cache row that will be replaced with new row.
+     */
+    public void store(GridCacheContext<?, ?> cctx, CacheDataRow newRow, @Nullable CacheDataRow prevRow,

Review comment:
       Replace IgniteIndexing (IndexSpi) with IndexProcessor




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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r601272268



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractLeafIO.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.cache.query.index.sorted.inline.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.ThreadLocalRowHandlerHolder;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+
+/**
+ * Leaf page to store index rows.
+ */
+public abstract class AbstractLeafIO extends BPlusLeafIO<IndexRow> implements InlineIO {
+    /**
+     * @param type Page type.
+     * @param ver Page format version.
+     * @param itemSize Single item size on page.
+     */
+    AbstractLeafIO(int type, int ver, int itemSize) {
+        super(type, ver, itemSize);
+    }
+

Review comment:
       empty line




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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r601270827



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/defragmentation/DefragIndexFactory.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.cache.query.index.sorted.defragmentation;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandlerFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.MetaPageInfo;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineRecommender;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.RootPage;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+
+/**
+ * Creates temporary index to defragment old index.
+ */
+public class DefragIndexFactory extends InlineIndexFactory {
+    /** Temporary offheap manager. */
+    private final IgniteCacheOffheapManager offheap;
+
+    /** Old index. */
+    private final InlineIndex oldIdx;
+
+    /** Temporary cache page memory. */
+    private final PageMemory newCachePageMemory;
+
+    /** */
+    private final InlineIndexRowHandlerFactory rowHndFactory;
+
+    /** */
+    public DefragIndexFactory(IgniteCacheOffheapManager offheap, PageMemory newCachePageMemory, InlineIndex oldIdx) {
+        // Row handler factory that produces no-op handler.
+        rowHndFactory = (def, settings) -> oldIdx.segment(0).rowHandler();
+
+        this.offheap = offheap;
+        this.oldIdx = oldIdx;
+        this.newCachePageMemory = newCachePageMemory;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected InlineIndexTree createIndexSegment(GridCacheContext<?, ?> cctx, SortedIndexDefinition def,
+        RootPage rootPage, IoStatisticsHolder stats, InlineRecommender recommender, int segmentNum) throws Exception {
+
+        InlineIndexTree tree = new InlineIndexTree(
+            def,
+            cctx,
+            def.treeName(),
+            offheap,
+            offheap.reuseListForIndex(def.treeName()),
+            newCachePageMemory,
+            // Use old row handler to have access to inline index key types.
+            pageIoResolver(),
+            rootPage.pageId().pageId(),
+            rootPage.isAllocated(),
+            oldIdx.inlineSize(),
+            def.keyTypeSettings(),
+            null,
+            stats,
+            rowHndFactory,
+            null
+        );
+
+        final MetaPageInfo oldInfo = oldIdx.segment(segmentNum).metaInfo();
+
+        tree.copyMetaInfo(oldInfo);
+
+        tree.enableSequentialWriteMode();
+
+        return tree;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RootPage rootPage(GridCacheContext<?, ?> ctx, String treeName, int segment) throws Exception {
+        return offheap.rootPageForIndex(ctx.cacheId(), treeName, segment);
+    }
+
+    /** */
+    private PageIoResolver pageIoResolver() {
+        return pageAddr -> {
+            PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+            if (io instanceof BPlusMetaIO)
+                return io;
+
+            //noinspection unchecked,rawtypes,rawtypes
+            return wrap((BPlusIO)io, rowHndFactory.create(null, null));
+        };
+    }
+
+    /** */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    static BPlusIO<IndexRow> wrap(BPlusIO<IndexRow> io, InlineIndexRowHandler rowHnd) {
+        assert io instanceof InlineIO;
+
+        if (io instanceof BPlusInnerIO) {
+            assert io instanceof AbstractInlineInnerIO
+                || io instanceof InlineInnerIO;
+
+            return new BPlusInnerIoDelegate((BPlusInnerIO<IndexRow>)io, rowHnd);
+        }
+        else {
+            assert io instanceof AbstractInlineLeafIO
+                || io instanceof InlineLeafIO;
+
+            return new BPlusLeafIoDelegate((BPlusLeafIO<IndexRow>)io, rowHnd);
+        }
+    }
+
+    /** */
+    private static <T extends BPlusIO<IndexRow> & InlineIO> IndexRow lookupRow(
+        InlineIndexRowHandler rowHnd,
+        long pageAddr,
+        int idx,
+        T io
+    ) {
+        long link = io.link(pageAddr, idx);
+
+        int off = io.offset(idx);
+
+        IndexKey[] keys = new IndexKey[rowHnd.indexKeyDefinitions().size()];
+
+        int fieldOff = 0;
+
+        for (int i = 0; i < rowHnd.inlineIndexKeyTypes().size(); i++) {
+            InlineIndexKeyType keyType = rowHnd.inlineIndexKeyTypes().get(i);
+
+            IndexKey key = keyType.get(pageAddr, off + fieldOff, io.inlineSize() - fieldOff);
+
+            fieldOff += keyType.inlineSize(key);
+
+            keys[i] = key;
+        }
+
+        return new IndexRowImpl(rowHnd, new CacheDataRowAdapter(link), keys);
+    }
+
+    /** */
+    private static class BPlusInnerIoDelegate<IO extends BPlusInnerIO<IndexRow> & InlineIO>
+        extends BPlusInnerIO<IndexRow> implements InlineIO {
+        /** */
+        private final IO io;
+
+        /** */
+        private final InlineIndexRowHandler rowHnd;
+
+        /** */
+        public BPlusInnerIoDelegate(IO io, InlineIndexRowHandler rowHnd) {

Review comment:
       public constructor for private inner class (the same below)




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r591814224



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/QueryIndexKeyDefinitionProvider.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.h2.index;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.internal.cache.query.index.NullsOrder;
+import org.apache.ignite.internal.cache.query.index.Order;
+import org.apache.ignite.internal.cache.query.index.SortOrder;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.h2.table.IndexColumn;
+
+/** Maps H2 columns to IndexKeyDefinition and InlineIndexKeyType. */
+public class QueryIndexKeyDefinitionProvider {
+    /** Table. */
+    private final GridH2Table table;
+
+    /** H2 index columns. */
+    private final List<IndexColumn> h2IdxColumns;
+
+    /** Unmodified list of index key definitions. */
+    private List<IndexKeyDefinition> keyDefs;
+
+    /** */
+    public QueryIndexKeyDefinitionProvider(GridH2Table table, List<IndexColumn> h2IdxColumns) {
+        this.table = table;
+        this.h2IdxColumns = h2IdxColumns;
+    }
+
+    /**
+     * @return List of index key definitions.
+     */
+    public List<IndexKeyDefinition> get() {

Review comment:
       let's rename it to `keyDefinitions`




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r582073753



##########
File path: modules/core/src/main/java/org/apache/ignite/cache/query/index/sorted/IndexKey.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents a complex index key.
+ */
+public interface IndexKey {

Review comment:
       Hierarchy was simplified.




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



[GitHub] [ignite] timoninmaxim commented on pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#issuecomment-805223851


   Hi @korlov42 ! I've updated the PR:
   1. H2RowCache is restored as IndexRowCache;
   2. InlineIndexColumnTest is restored. It helps me to find a bug, bad to miss this test from my side :(
   
   Could you please check my PR again?
   


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r579621637



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.SystemProperty;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRowImpl;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Write to a log recommendation for inline size.
+ */
+public class InlineRecommender {
+    /** @see #IGNITE_THROTTLE_INLINE_SIZE_CALCULATION */
+    public static final int DFLT_THROTTLE_INLINE_SIZE_CALCULATION = 1_000;
+
+    /** */
+    @SystemProperty(value = "How often real invocation of inline size calculation will be skipped.", type = Long.class,
+        defaults = "" + DFLT_THROTTLE_INLINE_SIZE_CALCULATION)
+    public static final String IGNITE_THROTTLE_INLINE_SIZE_CALCULATION = "IGNITE_THROTTLE_INLINE_SIZE_CALCULATION";
+
+    /** Counter of inline size calculation for throttling real invocations. */
+    private final ThreadLocal<Long> inlineSizeCalculationCntr = ThreadLocal.withInitial(() -> 0L);
+
+    /** How often real invocation of inline size calculation will be skipped. */
+    private final int inlineSizeThrottleThreshold =
+        IgniteSystemProperties.getInteger(IGNITE_THROTTLE_INLINE_SIZE_CALCULATION,
+            DFLT_THROTTLE_INLINE_SIZE_CALCULATION);
+
+    /** Keep max calculated inline size for current index. */
+    private final AtomicInteger maxCalculatedInlineSize = new AtomicInteger();
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Index definition. */
+    private final SortedIndexDefinition def;
+
+    /** Constructor. */
+    public InlineRecommender(GridCacheContext cctx, SortedIndexDefinition def) {
+        log = cctx.kernalContext().indexing().getLogger();
+        this.def = def;
+    }
+
+    /**
+     * Calculate aggregate inline size for given indexes and log recommendation in case calculated size more than
+     * current inline size.
+     */
+    @SuppressWarnings({"ConditionalBreakInInfiniteLoop", "IfMayBeConditional"})
+    public void recommend(IndexRow row, int currInlineSize) {
+        // Do the check only for put operations.
+        if (row instanceof IndexSearchRowImpl)
+            return;
+
+        Long invokeCnt = inlineSizeCalculationCntr.get();
+
+        inlineSizeCalculationCntr.set(++invokeCnt);
+
+        boolean throttle = invokeCnt % inlineSizeThrottleThreshold != 0;
+
+        if (throttle)
+            return;

Review comment:
       Hi, agree. Replaced with optimistic CAS. 




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r601297006



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateValueUtils.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+/**
+ * DateValue is a representation of a date in bit form:
+ *
+ * dv = (year << SHIFT_YEAR) | (month << SHIFT_MONTH) | day.
+ */
+public class DateValueUtils {

Review comment:
       Add private constructor. Rename DateValueUtils to DateValueConstants.




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



[GitHub] [ignite] korlov42 commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r590063542



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/DurableBackgroundCleanupIndexTreeTask.java
##########
@@ -102,7 +96,7 @@ public DurableBackgroundCleanupIndexTreeTask(
             IoStatisticsHolderIndex stats = new IoStatisticsHolderIndex(

Review comment:
       `stats` is not being used

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/Order.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.cache.query.index;
+
+/**
+ * Represents ordering of rows within sorted index.
+ */
+public class Order {
+    /** */
+    private final NullsOrder nullsOrder;
+
+    /** */
+    private final SortOrder sortOrder;
+
+    /** */
+    public Order(SortOrder sortOrder, NullsOrder nullsOrder) {
+        this.sortOrder = sortOrder;
+        this.nullsOrder = nullsOrder;
+    }
+
+    /** */
+    public SortOrder getSortOrder() {

Review comment:
       According to [our guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-Gettersandsetters) a prefix `get` should be omitted for internal packages. Please fix this everywhere in this PR.

##########
File path: modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
##########
@@ -84,7 +91,56 @@
      * @param expirationTime Expiration time or 0 if never expires.
      * @throws IgniteSpiException If failed.
      */
-    public void store(@Nullable String cacheName, Object key, Object val, long expirationTime) throws IgniteSpiException;
+    public default void store(@Nullable String cacheName, Object key, Object val, long expirationTime) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /**
+     * Updates index with new row. Note that key is unique for cache, so if cache contains multiple indexes
+     * the key should be removed from indexes other than one being updated.
+     *
+     * @param cctx Cache context.
+     * @param newRow cache row to store in index.
+     * @param prevRow optional cache row that will be replaced with new row.
+     */
+    public void store(GridCacheContext<?, ?> cctx, CacheDataRow newRow, @Nullable CacheDataRow prevRow,

Review comment:
        I don't sure it's a good idea to request an internal classes from public API

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.IndexSearchRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object get() {
+            return null;

Review comment:
       looks like this is the best we can do now. Let's add a comment

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/MetaPageInfo.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.lang.IgniteProductVersion;
+
+/**
+ * Meta page stores meta data about InlineIndexTree.
+ */
+public class MetaPageInfo {
+    /** Inline size used for a tree. */
+    int inlineSize;

Review comment:
       let's add `private` to all fields

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.cache.query.index.sorted.keys;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+
+/** */
+public class BooleanIndexKey implements IndexKey {
+    /** */
+    private final boolean key;
+
+    /** */
+    public BooleanIndexKey(boolean key) {
+        this.key = key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getKey() {
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getType() {
+        return IndexKeyTypes.BOOLEAN;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(IndexKey o, IndexKeyTypeSettings keySettings) {

Review comment:
       it looks like `keySettings` is used only for `BytesIndexKey`. So may be it worth to provide `SignedBytesIndexKey` similar to `SignedBytesInlineIndexKeyType`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/JavaObjectKeySerializer.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.Marshaller;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Serializer for representing JO as byte array in inline.
+ */
+public class JavaObjectKeySerializer {
+    /** Class loader. */
+    private final ClassLoader clsLdr;
+
+    /** Marshaller. */
+    private final Marshaller marshaller;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Ignite configuration.
+     */
+    public JavaObjectKeySerializer(@NotNull IgniteConfiguration cfg) {

Review comment:
       we don't use `@NotNull` annotation

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/DurableBackgroundCleanupIndexTreeTask.java
##########
@@ -46,53 +49,44 @@
     private List<Long> rootPages;
 
     /** */
-    private transient List<H2Tree> trees;
+    private transient List<InlineIndexTree> trees;
 
     /** */
     private transient volatile boolean completed;
 
     /** */
-    private String cacheGrpName;
+    private final String cacheName;
 
     /** */
-    private String cacheName;
+    private final IndexName idxName;

Review comment:
       I would prefer to leave idxName as a simple string. This will allow us to remove Serialized from IndexName. I think this is a good idea because the current approach to naming system indexes with a constant (like `PK_IDX_NAME`) violates some sql restrictions (it is currently possible to have multiple indexes with the same name in the same sql schema). This should be reworked, and using `IndexName` as being serializable could potentially lead to a compatibility issue.
   
   So WDYT?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/Index.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for Ignite indexes.
+ */
+public interface Index {
+    /**
+     * Unique ID.
+     */
+    public UUID id();
+
+    /**
+     * Index name.
+     */
+    public String name();
+
+    /**
+     * Checks whether index handles specified cache row.
+     *
+     * @param row Cache row.
+     * @return Whether index handles specified cache row
+     */
+    public boolean canHandle(CacheDataRow row) throws IgniteCheckedException;
+
+    /**
+     * Callback that runs when the underlying cache is updated.
+     *
+     * @param oldRow Cache row that was replaced with newRow.
+     * @param newRow Cache row that was stored.
+     * @param prevRowAvailable Whether oldRow available.
+     */
+    public void onUpdate(@Nullable CacheDataRow oldRow, @Nullable CacheDataRow newRow, boolean prevRowAvailable)
+        throws IgniteCheckedException;
+
+    /**
+     * Put cache row to index.
+     *
+     * @return {@code True} if replaced existing row.
+     */
+    public boolean putx(CacheDataRow row) throws IgniteCheckedException;
+
+    /**
+     * Remove cache row from index.
+     *
+     * @return {@code True} if removed row.
+     */
+    public boolean removex(CacheDataRow row) throws IgniteCheckedException;

Review comment:
       let's remove `putx` and `removex` from interface and from implementations too.
   
   It looks like `removex` actually is not called anywhere, and `putx` is called in exactly one place and could be safely replaced with `onUpdate(null, row, false)`
   

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java
##########
@@ -1,163 +0,0 @@
-/*
- * 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.h2;
-
-import java.util.Iterator;
-import java.util.Map;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.PageIdUtils;
-import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.processors.cache.CacheGroupContext;
-import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
-import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
-import org.apache.ignite.internal.util.typedef.F;
-import org.jsr166.ConcurrentLinkedHashMap;
-
-import static org.jsr166.ConcurrentLinkedHashMap.DFLT_INIT_CAP;
-import static org.jsr166.ConcurrentLinkedHashMap.DFLT_LOAD_FACTOR;
-
-/**
- * H2 row cache.
- */
-public class H2RowCache implements GridQueryRowCacheCleaner {

Review comment:
       Why did you decide to get rid of this? This row cache is not the same thing as H2's internal object cache

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesCompareUtils.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.cache.query.index.sorted.keys;
+
+/** */
+public final class BytesCompareUtils {
+    /** */
+    public static int compareNotNullSigned(byte[] arr0, byte[] arr1) {
+        if (arr0 == arr1)
+            return 0;
+        else {
+            int commonLen = Math.min(arr0.length, arr1.length);
+
+            for (int i = 0; i < commonLen; ++i) {
+                byte b0 = arr0[i];
+                byte b1 = arr1[i];
+
+                if (b0 != b1)
+                    return b0 > b1 ? 1 : -1;
+            }
+
+            return Integer.signum(arr0.length - arr1.length);
+        }
+    }
+
+    /** */
+    public static int compareNotNullUnsigned(byte[] arr0, byte[] arr1) {
+        if (arr0 == arr1)
+            return 0;
+        else {
+            int commonLen = Math.min(arr0.length, arr1.length);
+
+            for (int i = 0; i < commonLen; ++i) {
+                int var4 = arr0[i] & 255;

Review comment:
       do var[1-3] exist?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexSearchRow.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+
+/**
+ * Represents a complex index key.
+ */
+public interface IndexSearchRow {

Review comment:
       the value of this interface is unclear to me. Seem like it could be simply merged with `IndexRow`. And it's better to replace `public IndexKey[] getKeys()` with `public int size()` 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IgniteIndexing.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.IgniteSpiContext;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.apache.ignite.spi.indexing.IndexingSpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of IndexingSpi that tracks all cache indexes.
+ */
+public class IgniteIndexing implements IndexingSpi {
+    /**
+     * Register inline IOs for sorted indexes.
+     */
+    static {
+        PageIO.registerH2(InnerIO.VERSIONS, LeafIO.VERSIONS, null, null);
+
+        AbstractInlineInnerIO.register();
+        AbstractInlineLeafIO.register();
+    }
+
+    /**
+     * Registry of all indexes. High key is a cache name, lower key is an unique index name.
+     */
+    private final Map<String, Map<String, Index>> cacheToIdx = new ConcurrentHashMap<>();
+
+    /**
+     * Registry of all index definitions. Key is {@link Index#id()}, value is IndexDefinition used for creating index.
+     */
+    private final Map<UUID, IndexDefinition> idxDefs = new ConcurrentHashMap<>();
+
+    /** Exclusive lock for DDL operations. */
+    private final ReentrantReadWriteLock ddlLock = new ReentrantReadWriteLock();
+
+    /** {@inheritDoc} */
+    @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
+        @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(@Nullable String cacheName, Object key, Object val,
+        long expirationTime) throws IgniteSpiException {
+        throw new IgniteSpiException("Not implemented.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(GridCacheContext<?, ?> cctx, CacheDataRow newRow, @Nullable CacheDataRow prevRow,
+        boolean prevRowAvailable)
+        throws IgniteSpiException {
+        try {
+            updateIndexes(cctx.name(), newRow, prevRow, prevRowAvailable);
+
+        } catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to store row in cache", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(Collection<? extends Index> idxs, CacheDataRow newRow, @Nullable CacheDataRow prevRow,
+        boolean prevRowAvailable) throws IgniteSpiException {
+        IgniteCheckedException err = null;
+
+        ddlLock.readLock().lock();
+
+        try {
+            for (Index idx : idxs)
+                err = updateIndex(idx, newRow, prevRow, prevRowAvailable, err);
+
+            if (err != null)
+                throw err;
+
+        } catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to store row in index", e);
+
+        } finally {
+            ddlLock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(String cacheName, @Nullable CacheDataRow prevRow) throws IgniteSpiException {
+        try {
+            updateIndexes(cacheName, null, prevRow, true);
+
+        } catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to remove row in cache", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Index createIndex(GridCacheContext<?, ?> cctx, IndexFactory factory, IndexDefinition definition) {
+        ddlLock.writeLock().lock();
+
+        try {
+            String cacheName = definition.getIdxName().cacheName();
+
+            cacheToIdx.putIfAbsent(cacheName, new ConcurrentHashMap<>());
+
+            String uniqIdxName = definition.getIdxName().fullName();
+
+            // GridQueryProcessor already checked schema operation for index duplication.
+            assert cacheToIdx.get(cacheName).get(uniqIdxName) == null : "Duplicated index name " + uniqIdxName;
+
+            Index idx = factory.createIndex(cctx, definition);
+
+            cacheToIdx.get(cacheName).put(uniqIdxName, idx);
+
+            idxDefs.put(idx.id(), definition);
+
+            return idx;
+
+        } finally {
+            ddlLock.writeLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeIndex(GridCacheContext<?, ?> cctx, IndexDefinition def, boolean softDelete) {

Review comment:
       `IndexName` should uniquely describe index, so let's use it instead of `IndexDefinition`

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyTypeRegistryTest.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IntegerIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.NullIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.PlainJavaObjectIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.StringIndexKey;
+import org.junit.Test;
+
+/** */
+public class InlineIndexKeyTypeRegistryTest {
+    /** */
+    private static final IndexKeyTypeSettings pojoHashKeyTypeSettings =
+        new IndexKeyTypeSettings();
+
+    /** */
+    private static final IndexKeyTypeSettings pojoArrayKeyTypeSettings =
+        new IndexKeyTypeSettings().inlineObjHash(false);
+
+    /** */
+    private static final IndexKeyTypeSettings strNoCompareKeyTypeSettings =
+        new IndexKeyTypeSettings().inlineObjHash(false).stringOptimizedCompare(false);
+
+    /** */
+    @Test
+    public void testNulls() {
+        int t = InlineIndexKeyTypeRegistry.get(NullIndexKey.INSTANCE, IndexKeyTypes.INT, pojoArrayKeyTypeSettings).type();
+        assert t == IndexKeyTypes.INT;

Review comment:
       let's replace `assert` with JUnit assertions




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



[GitHub] [ignite] korlov42 commented on pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
korlov42 commented on pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#issuecomment-796082130


   Hi, @timoninmaxim!
   
   > Cursor implementations can't be implemented other way 
   
   I'm OK with this
   
   > MVCC tests, including C++ mvcc test, can I just remove all of them?
   
   I think it's better to discuss with community
   
   > If design is OK, then I will run performance tests.
   
   Some refactoring still needed -- `IndexingSpi` should not accept classes from internal packages as methods params.
   
   Also a few test classes were removed besides MVCC tests, but no replacement were provided. It's better to restore them.


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r592237116



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java
##########
@@ -1,163 +0,0 @@
-/*
- * 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.h2;
-
-import java.util.Iterator;
-import java.util.Map;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.PageIdUtils;
-import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.processors.cache.CacheGroupContext;
-import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
-import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
-import org.apache.ignite.internal.util.typedef.F;
-import org.jsr166.ConcurrentLinkedHashMap;
-
-import static org.jsr166.ConcurrentLinkedHashMap.DFLT_INIT_CAP;
-import static org.jsr166.ConcurrentLinkedHashMap.DFLT_LOAD_FACTOR;
-
-/**
- * H2 row cache.
- */
-public class H2RowCache implements GridQueryRowCacheCleaner {

Review comment:
       This was discussed with @tledkov-gridgain. It was an optimization for some specific corner cases for a specific GG client. Decided to remove this code as obsolete.




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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #8490: IGNITE-13056 Move indexes to core

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #8490:
URL: https://github.com/apache/ignite/pull/8490#discussion_r601268225



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/SingleCursor.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check wherther value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /**
+     * Note that this implimentation violates the contract of GridCusror. It must be "before first" but this implementation

Review comment:
       typo: 'implimentation'




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