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/11/04 09:44:16 UTC

[GitHub] [ignite-3] korlov42 commented on a diff in pull request #1205: IGNITE-17748 Enrich InternalTable.scan API in order to support index scans.

korlov42 commented on code in PR #1205:
URL: https://github.com/apache/ignite-3/pull/1205#discussion_r1013747873


##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/HashIndexDescriptor.java:
##########
@@ -49,6 +49,25 @@ public static class HashIndexColumnDescriptor implements ColumnDescriptor {
 
         private final boolean nullable;
 
+        /**
+         * Creates a Column Descriptor.
+         *
+         * @param name Name of the column.
+         * @param type Type of the column.
+         * @param nullable Flag indicating that the column may contain {@code null}s.
+         */
+        public HashIndexColumnDescriptor(String name, NativeType type, boolean nullable) {
+            this.name = name;
+            this.type = type;
+            this.nullable = nullable;
+        }
+
+        /**
+         * Creates a Column Descriptor.
+         *
+         * @param tableColumnView Table column configuration.
+         * @param indexColumnView Index column configuration.

Review Comment:
   constructor has only one param 



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/IgniteIndexScan.java:
##########
@@ -37,6 +38,9 @@
 public class IgniteIndexScan extends AbstractIndexScan implements SourceAwareIgniteRel {
     private final long sourceId;
 
+    /** Index collation. Required only for rewriting index scan to table scan + sort in case of index rebuild. */

Review Comment:
   Misleading comment. This collation is used to properly expand search bounds as well as to create index row converter 



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/IndexScanNode.java:
##########
@@ -74,8 +79,12 @@
 
     private final @Nullable Function<RowT, RowT> rowTransformer;
 
+    private final IgniteTetraFunction<ExecutionContext<RowT>, BinaryRow, RowFactory<RowT>, ImmutableBitSet, RowT> tableRowConverter;

Review Comment:
   I believe it worth to introduce interface/class with meaningful name



##########
modules/index/build.gradle:
##########
@@ -24,6 +24,7 @@ dependencies {
     implementation project(':ignite-core')
     implementation project(':ignite-configuration')
     implementation project(':ignite-schema')
+    implementation project(':ignite-table')

Review Comment:
   looks like dependency on table module have been already added. Worth to revert this change as well as one below



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java:
##########
@@ -150,6 +163,11 @@ public class PartitionReplicaListener implements ReplicaListener {
     /** Placement Driver. */
     private final PlacementDriver placementDriver;
 
+    /** Runs async scan tasks for effective tail recursion execution (avoid deep recursive calls). */
+    private final Executor scanRequestExecutor = Executors.newSingleThreadExecutor(
+            new NamedThreadFactory("scan-query-executor-", Loggers.forClass(PartitionReplicaListener.class))

Review Comment:
   we need to use `org.apache.ignite.internal.thread.NamedThreadFactory#threadPrefix` for thread prefix. Also executor have to be stopped at the stop of the table



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/IndexScanNode.java:
##########
@@ -33,12 +32,17 @@
 import java.util.function.Function;
 import java.util.function.Predicate;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.ImmutableBitSet;

Review Comment:
   we are trying to avoid using guava's collection in our codebase



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