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 2022/07/04 14:46:16 UTC

[GitHub] [ignite] timoninmaxim commented on a diff in pull request #10080: IGNITE-17046 Move H2RowComparator logic to the core module

timoninmaxim commented on code in PR #10080:
URL: https://github.com/apache/ignite/pull/10080#discussion_r912962497


##########
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCompartorImpl.java:
##########
@@ -81,15 +81,22 @@ private boolean sameType(IndexKey key, int idxType) {
     }
 
     /** */
-    private int compare(IndexKey lkey, IndexKey rkey) {
-        if (lkey == NullIndexKey.INSTANCE)
-            return lkey.compare(rkey);
-        else if (rkey == NullIndexKey.INSTANCE)
-            return 1;
-
-        if (lkey.type() == rkey.type())
-            return lkey.compare(rkey);
+    private int compare(IndexKey lkey, IndexKey rkey) throws IgniteCheckedException {
+        try {
+            if (lkey == NullIndexKey.INSTANCE)
+                return lkey.compare(rkey);
+            else if (rkey == NullIndexKey.INSTANCE)
+                return 1;
+
+            if (lkey.isComparableTo(rkey))
+                return lkey.compare(rkey);
+            else if (rkey.isComparableTo(lkey))
+                return -rkey.compare(lkey);
+        }
+        catch (RuntimeException e) {

Review Comment:
   Why do we need handle runtime exceptions here?



##########
modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java:
##########
@@ -265,7 +265,7 @@ private H2PkHashIndexCursor(Iterator<GridCursor<? extends CacheDataRow>> iter) {
         /** {@inheritDoc} */
         @Override public Row get() {
             try {
-                return desc.createRow(curr.get());
+                return new H2CacheRow(desc, curr.get());
             }
             catch (IgniteCheckedException e) {

Review Comment:
   No exceptions now



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowDescriptor.java:
##########
@@ -56,13 +48,13 @@ public class GridH2RowDescriptor {
     private volatile String[] fields;
 
     /** */
-    private volatile int[] fieldTypes;
+    private volatile IndexKeyType[] fieldTypes;

Review Comment:
   Let's rename IndexKeyType to FieldType or smth similar. This collection isn't about index fields.



##########
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexRowHandler.java:
##########
@@ -54,31 +49,44 @@ public class QueryIndexRowHandler implements InlineIndexRowHandler {
     /** Index key type settings. */
     private final IndexKeyTypeSettings keyTypeSettings;
 
-    /** H2 Table. */
-    private final GridH2Table h2Table;
-
     /** */
-    public QueryIndexRowHandler(GridH2Table h2table, List<IndexColumn> h2IdxColumns,
-        LinkedHashMap<String, IndexKeyDefinition> keyDefs, List<InlineIndexKeyType> keyTypes, IndexKeyTypeSettings keyTypeSettings) {
-        this.h2IdxColumns = h2IdxColumns;
-        this.keyTypes = keyTypes;
+    public QueryIndexRowHandler(
+        GridQueryRowDescriptor rowDescriptor,
+        LinkedHashMap<String, IndexKeyDefinition> keyDefs,
+        List<InlineIndexKeyType> keyTypes,
+        IndexKeyTypeSettings keyTypeSettings
+    ) {
+        this.keyTypes = Collections.unmodifiableList(keyTypes);
         this.keyDefs = Collections.unmodifiableList(new ArrayList<>(keyDefs.values()));
-        this.h2Table = h2table;
-        cacheDesc = h2table.rowDescriptor();
+        this.rowDescriptor = rowDescriptor;
         this.keyTypeSettings = keyTypeSettings;
+
+        List<Integer> keyColumns = new ArrayList<>(keyDefs.size());
+        List<String> fields = new ArrayList<>(rowDescriptor.type().fields().keySet());
+
+        for (String fieldName : keyDefs.keySet()) {
+            if (QueryUtils.KEY_FIELD_NAME.equals(fieldName))

Review Comment:
   Let's move this logic inside `GridQueryRowDescriptor`, it already have similar methods. Also let's move logic of `QueryIndexRowHandler#getKey` to the descriptor too.



##########
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java:
##########
@@ -527,14 +529,35 @@ public void destroyOrphanIndex(
         tree.destroy();
     }
 
+    /**
+     * @return {@code true} In case of use an unwrapped PK for the index.
+     */
+    public boolean useUnwrappedPk(GridCacheContext<?, ?> cctx, String treeName) {
+        IgniteCacheDatabaseSharedManager db = cctx.shared().database();
+        db.checkpointReadLock();
+
+        try {
+            RootPage page = cctx.offheap().findRootPageForIndex(cctx.cacheId(), treeName, 0);
+
+            return page == null || metaPageInfo(page, cctx.groupId(), cctx.dataRegion().pageMemory()).useUnwrappedPk();
+        }
+        catch (IgniteCheckedException ignore) {
+            return true;
+        }
+        finally {
+            db.checkpointReadUnlock();
+        }
+
+    }
+
     /**
      * @param page Root page.
      * @param grpId Cache group id.
      * @param pageMemory Page memory.
-     * @return Inline size.
+     * @return Meta page info.
      * @throws IgniteCheckedException If something went wrong.
      */
-    private int inlineSize(RootPage page, int grpId, PageMemory pageMemory) throws IgniteCheckedException {
+    private MetaPageInfo metaPageInfo(RootPage page, int grpId, PageMemory pageMemory) throws IgniteCheckedException {

Review Comment:
   Currently we have 2 similar methods for getting MetaPageInfo. One is here `IndexProcessor#metaPageInfo`, and another one is in `InlineIndexTree#metaInfo`. Let's use single one instead.



##########
modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2ValueIndexKey.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
+import org.apache.ignite.internal.processors.query.h2.H2Utils;
+import org.h2.table.Table;
+import org.h2.value.Value;
+
+/**
+ * Index key wrapped over H2 Value.
+ */
+public class H2ValueIndexKey implements IndexKey {
+    /** */
+    private final CacheObjectValueContext coCtx;
+
+    /** */
+    private final Table table;
+
+    /** */
+    private final Value val;
+
+    /** */
+    public H2ValueIndexKey(CacheObjectValueContext coCtx, Table table, Value val) {
+        this.coCtx = coCtx;
+        this.table = table;
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object key() {
+        return val.getObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IndexKeyType type() {
+        return IndexKeyType.forCode(val.getType());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(IndexKey o) {
+        try {
+            return compareValues(val, H2Utils.wrap(coCtx, o.key(), o.type().code()));
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * @param v1 First value.
+     * @param v2 Second value.
+     * @return Comparison result.
+     */
+    public int compareValues(Value v1, Value v2) {

Review Comment:
   private



##########
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/IndexKey.java:
##########
@@ -29,12 +29,19 @@ public interface IndexKey {
     public Object key();
 
     /**
-     * @return Index key type {@link IndexKeyTypes}.
+     * @return Index key type {@link IndexKeyType}.
      */
-    public int type();
+    public IndexKeyType type();
 
     /**
      * @return Comparison result with other IndexKey the same type.

Review Comment:
   Now `the same type` isn't true. Let's fix the docs



##########
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/BooleanInlineIndexKeyType.java:
##########
@@ -46,10 +48,15 @@ public BooleanInlineIndexKeyType() {
     }
 
     /** {@inheritDoc} */
-    @Override public int compare0(long pageAddr, int off, BooleanIndexKey key) {
+    @Override public boolean isComparableTo(IndexKey key) {
+        return key instanceof NumericIndexKey;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, IndexKey key) {
         boolean bool1 = PageUtils.getByte(pageAddr, off + 1) != 0;
 
-        return Integer.signum(Boolean.compare(bool1, (boolean)key.key()));
+        return -((NumericIndexKey)key).compareTo(bool1);

Review Comment:
   It's required here to return `Integer.signum()`. `Comparable#compareTo` doesn't specify return values, it just describe them as "positive" and "negative". But we have different logic in inline comparison for values "-1" and "-2". 



##########
modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/BooleanInlineIndexKeyType.java:
##########
@@ -46,10 +48,15 @@ public BooleanInlineIndexKeyType() {
     }
 
     /** {@inheritDoc} */
-    @Override public int compare0(long pageAddr, int off, BooleanIndexKey key) {
+    @Override public boolean isComparableTo(IndexKey key) {
+        return key instanceof NumericIndexKey;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, IndexKey key) {
         boolean bool1 = PageUtils.getByte(pageAddr, off + 1) != 0;
 
-        return Integer.signum(Boolean.compare(bool1, (boolean)key.key()));
+        return -((NumericIndexKey)key).compareTo(bool1);

Review Comment:
   see `ObjectHashInlineIndexKeyType#compare0`.



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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