You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "lowka (via GitHub)" <gi...@apache.org> on 2023/06/15 12:18:20 UTC

[GitHub] [ignite-3] lowka opened a new pull request, #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

lowka opened a new pull request, #2199:
URL: https://github.com/apache/ignite-3/pull/2199

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

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

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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236786310


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/NoOpTransaction.java:
##########
@@ -43,14 +43,32 @@ public final class NoOpTransaction implements InternalTransaction {
 
     private final TablePartitionId groupId = new TablePartitionId(1, 0);
 
+    private final boolean readOnly;
+
     /**
-     * Constructs the object.
+     * Constructs a read only transaction.
      *
      * @param name Name of the node.
      */
     public NoOpTransaction(String name) {
+        this(name, true);
+    }
+
+    /**
+     * Constructs a transaction.
+     *
+     * @param name Name of the node.
+     * @param readOnly Read-only or not.
+     */
+    public NoOpTransaction(String name, boolean readOnly) {

Review Comment:
   ```suggestion
       public static NoOpTransation readOnly(String name) {
               return new NoOpTransation(name, true);
       }
       
      /**
      * Constructs a read-write transaction.
      *
      * @param name Name of the node.
      */
      public static NoOpTransation readWrite(String name) {
               return new NoOpTransation(name, false);
       }
       
       private NoOpTransaction(String name, boolean readOnly) {
   ```



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableCallback.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.sql.engine.exec;
+
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+
+/**
+ * Callback that is called when {@link ExecutableTable} is loaded.
+ */
+@FunctionalInterface
+public interface ExecutableTableCallback {

Review Comment:
   I didn't find this callback is ever used in production code. 
   The callback looks like a test stuff, let's drop it.
   The single place in tests, where it used, can be rewritten with Mockito.spy.



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/PrimaryReplica.java:
##########
@@ -57,4 +59,26 @@ public ClusterNode node() {
     public long term() {
         return term;
     }
+
+    @Override

Review Comment:
   ```suggestion
       /** {@inheritDocs} */
       @Override
   ```



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/PrimaryReplica.java:
##########
@@ -57,4 +59,26 @@ public ClusterNode node() {
     public long term() {
         return term;
     }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        PrimaryReplica that = (PrimaryReplica) o;
+        return term == that.term && Objects.equals(node, that.node);
+    }
+
+    @Override

Review Comment:
   ```suggestion
       /** {@inheritDocs} */
       @Override
   ```



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java:
##########
@@ -293,6 +297,10 @@ public synchronized void start() {
         services.forEach(LifecycleAware::start);
     }
 
+    public ExecutionDependencyResolver dependencyResolver() {

Review Comment:
   ```suggestion
       @TestOnly
       public ExecutionDependencyResolver dependencyResolver() {
   ```



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistrySelfTest.java:
##########
@@ -79,9 +88,10 @@ public void testGetTable() {
         CompletableFuture<ExecutableTable> f = tester.getTable(tableId);
         ExecutableTable executableTable = f.join();
 
-        assertNotNull(executableTable.scanableTable());
+        assertNotNull(executableTable.scannableTable());
         assertNotNull(executableTable.updatableTable());
-        assertNotNull(executableTable.rowConverter());
+
+        verify(callback).onTableLoaded(eq(executableTable), eq(TABLE_NAME), eq(descriptor));

Review Comment:
   You can use Mockito.spy on `registry` object instead of `callback`.



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/PrimaryReplica.java:
##########
@@ -57,4 +59,26 @@ public ClusterNode node() {
     public long term() {
         return term;
     }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        PrimaryReplica that = (PrimaryReplica) o;
+        return term == that.term && Objects.equals(node, that.node);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(node, term);
+    }
+
+    @Override

Review Comment:
   ```suggestion
       /** {@inheritDocs} */
       @Override
   ```



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: [WIP] IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1238269338


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/IndexScanNodeTest.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.stream.Collectors;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.index.ColumnCollation;
+import org.apache.ignite.internal.index.HashIndex;
+import org.apache.ignite.internal.index.IndexDescriptor;
+import org.apache.ignite.internal.index.SortedIndexDescriptor;
+import org.apache.ignite.internal.index.SortedIndexImpl;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.IgniteIndex;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+/**
+ * Test {@link IndexScanNode} execution.
+ */
+public class IndexScanNodeTest extends AbstractExecutionTest {
+
+    /**
+     * Sorted index scan execution.
+     */
+    @Test
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void testSortedIndex() {
+        IgniteIndex index = newSortedIndex();
+
+        TestScannableTable<Object[]> scannableTable = new TestScannableTable<>();
+        scannableTable.setPartitionData(0, new Object[]{4}, new Object[]{5});
+        scannableTable.setPartitionData(2, new Object[]{1}, new Object[]{2});
+
+        Comparator<Object[]> cmp = (a, b) -> {
+            Comparable o1 = (Comparable<?>) a[0];
+            Comparable o2 = (Comparable<?>) b[0];
+            return o1.compareTo(o2);
+        };
+
+        Tester tester = new Tester();
+
+        IndexScanNode<Object[]> node = tester.createSortedIndex(index, scannableTable, cmp);
+        List<Object[]> result = tester.execute(node);
+
+        tester.expectResult(result, new Object[]{1}, new Object[]{2}, new Object[]{4}, new Object[]{5});
+    }
+
+    /**
+     * Hash index lookup execution.
+     */
+    @Test
+    public void testHashIndex() {
+        IgniteIndex index = newHashIndex();
+
+        TestScannableTable<Object[]> scannableTable = new TestScannableTable<>();
+        scannableTable.setPartitionData(0, new Object[]{2}, new Object[]{1});
+        scannableTable.setPartitionData(2, new Object[]{0});
+
+        Tester tester = new Tester();
+
+        IndexScanNode<Object[]> node = tester.createHashIndex(index, scannableTable);
+        List<Object[]> result = tester.execute(node);
+
+        tester.expectResult(result, new Object[]{2}, new Object[]{1}, new Object[]{0});
+    }
+
+    private static IgniteIndex newHashIndex() {
+        IndexDescriptor descriptor = new IndexDescriptor("IDX", List.of("C1"));
+        HashIndex index = new HashIndex(1, Mockito.mock(InternalTable.class), descriptor);
+
+        return new IgniteIndex(index);
+    }
+
+    private static IgniteIndex newSortedIndex() {
+        List<String> columnNames = List.of("C1");
+        List<ColumnCollation> columnCollations = List.of(ColumnCollation.ASC_NULLS_LAST);
+        SortedIndexDescriptor descriptor = new SortedIndexDescriptor("IDX", columnNames, columnCollations);
+        SortedIndexImpl index = new SortedIndexImpl(1, Mockito.mock(InternalTable.class), descriptor);
+
+        return new IgniteIndex(index);
+    }
+
+    private class Tester {
+
+        private final ExecutionContext<Object[]> ctx;
+
+        Tester() {
+            this.ctx = executionContext();
+        }
+
+        IndexScanNode<Object[]> createSortedIndex(IgniteIndex index, TestScannableTable<?> scannableTable, Comparator<Object[]> cmp) {
+            return createIndexNode(ctx, index, scannableTable, cmp);
+        }
+
+        IndexScanNode<Object[]> createHashIndex(IgniteIndex index, TestScannableTable<?> scannableTable) {
+            return createIndexNode(ctx, index, scannableTable, null);
+        }
+
+        List<Object[]> execute(IndexScanNode<Object[]> indexNode) {
+            RootNode<Object[]> root = new RootNode<>(ctx);
+
+            root.register(indexNode);
+
+            List<Object[]> actual = new ArrayList<>();
+            while (root.hasNext()) {
+                Object[] row = root.next();
+                actual.add(row);
+            }
+
+            root.close();
+
+            return actual;
+        }
+
+        void expectResult(List<Object[]> actual, Object[]... expected) {

Review Comment:
   ```suggestion
           static void validateResult(List<Object[]> actual, Object[]... expected) {
   ```
   Also, I think `actual` and `expected` should be of the same type: either both List or arrays.



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239908411


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,961 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RW");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RO");

Review Comment:
   ```suggestion
        static Stream<Arguments> transactions() {
           return Stream.of(
                   Arguments.of(Named.of("Read-only transaction", NoOpTransaction.readOnly("RO"))),
                   Arguments.of(Named.of(Read-write transaction", NoOpTransaction.readWrite("RW"))),
           );
       }
   ```



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236859774


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistrySelfTest.java:
##########
@@ -79,9 +88,10 @@ public void testGetTable() {
         CompletableFuture<ExecutableTable> f = tester.getTable(tableId);
         ExecutableTable executableTable = f.join();
 
-        assertNotNull(executableTable.scanableTable());
+        assertNotNull(executableTable.scannableTable());
         assertNotNull(executableTable.updatableTable());
-        assertNotNull(executableTable.rowConverter());
+
+        verify(callback).onTableLoaded(eq(executableTable), eq(TABLE_NAME), eq(descriptor));

Review Comment:
   I think `Mockito.spy` should be used for partial mocks (see Mockito docs) in some pathological cases of  legacy code when one can not even provide an adapter. 
   So I see no reason to use it here.
   



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236717349


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java:
##########
@@ -63,6 +72,113 @@ public <RowT> Publisher<RowT> scan(ExecutionContext<RowT> ctx, PartitionWithTerm
             pub = internalTable.scan(partWithTerm.partId(), txAttributes.id(), recipient, null, null, null, 0, null);
         }
 
-        return StorageScanNode.convertPublisher(pub, (item) -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
+        return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
     }
+
+    /** {@inheritDoc} */
+    @Override
+    public <RowT> Publisher<RowT> indexRangeScan(ExecutionContext<RowT> ctx, PartitionWithTerm partWithTerm, RowFactory<RowT> rowFactory,
+            int indexId, String indexName, List<String> columns, @Nullable RangeCondition<RowT> cond, @Nullable BitSet requiredColumns) {

Review Comment:
   btw, `indexName` is not used



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239908411


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,961 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RW");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RO");

Review Comment:
   ```suggestion
        static Stream<Arguments> transactions() {
           return Stream.of(
                   Arguments.of(NoOpTransaction.readOnly("RO")),
                   Arguments.of(NoOpTransaction.readWrite("RW")),
           );
       }
   ```



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1242013058


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,952 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RO");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RW");
+
+    private static final IgniteTypeFactory TYPE_FACTORY = Commons.typeFactory();
+
+    @Mock(lenient = true)
+    private InternalTable internalTable;
+
+    @Mock(lenient = true)
+    private ExecutionContext<Object[]> ctx;
+
+    @Mock
+    private BinaryRow binaryRow;
+
+    /**
+     * Table scan.
+     */
+    @ParameterizedTest
+    @MethodSource("transactions")
+    public void testTableScan(NoOpTransaction tx) {
+        TestInput data = new TestInput();
+        data.addRow(binaryRow);
+
+        Tester tester = new Tester(data);
+
+        int partitionId = 1;
+        long term = 2;
+
+        ResultCollector collector = tester.tableScan(partitionId, term, tx);
+
+        if (tx.isReadOnly()) {
+            HybridTimestamp timestamp = tx.readTimestamp();
+            ClusterNode clusterNode = tx.clusterNode();
+
+            verify(internalTable).scan(partitionId, timestamp, clusterNode);
+        } else {
+            ClusterNode clusterNode = tx.clusterNode();
+
+            verify(internalTable).scan(partitionId, tx.id(), new PrimaryReplica(clusterNode, term), null, null, null, 0, null);
+        }
+
+        data.sendRows();
+        data.done();
+
+        collector.expectRow(binaryRow);
+        collector.expectCompleted();
+    }
+
+    /**
+     * Table scan error propagation.
+     */
+    @ParameterizedTest
+    @MethodSource("transactions")
+    public void testTableScanError(NoOpTransaction tx) {
+        TestInput input = new TestInput();
+        input.addRow(binaryRow);
+
+        Tester tester = new Tester(input);
+
+        int partitionId = 1;
+        long term = 2;
+
+        ResultCollector collector = tester.tableScan(partitionId, term, tx);
+
+        input.sendRows();
+
+        RuntimeException err = new RuntimeException("err");
+        input.sendError(err);
+
+        collector.expectRow(binaryRow);
+        collector.expectError(err);
+    }
+
+    /**
+     * Index scan with different bounds.
+     */
+    @ParameterizedTest
+    @CsvSource({
+            // RO
+            "true, INCLUSIVE, NONE",
+            "true, EXCLUSIVE, NONE",
+            "true, NONE, INCLUSIVE",
+            "true, NONE, EXCLUSIVE",
+            "true, INCLUSIVE, INCLUSIVE",
+            "true, INCLUSIVE, EXCLUSIVE",
+            "true, EXCLUSIVE, INCLUSIVE",
+            "true, EXCLUSIVE, EXCLUSIVE",
+            // RW
+            "false, INCLUSIVE, NONE",
+            "false, EXCLUSIVE, NONE",
+            "false, NONE, INCLUSIVE",
+            "false, NONE, EXCLUSIVE",
+            "false, INCLUSIVE, INCLUSIVE",
+            "false, INCLUSIVE, EXCLUSIVE",
+            "false, EXCLUSIVE, INCLUSIVE",
+            "false, EXCLUSIVE, EXCLUSIVE",
+    })

Review Comment:
   ```suggestion
       static Stream<Arguments> indexScanParameters() {
           List<Arguments> args = new ArrayList<>();
           
           for(Bound leftBound : Bound.values) {
               for(Bound rightBound : Bound.values) {
                   arguments.add(Argtments.of(NoOpTransaction.readOnly("RO"), leftBound, rightBound));
                   arguments.add(Argtments.of(NoOpTransaction.readWrite("RW"), leftBound, rightBound));
               }
               
               return args.stream();
           }
       }
       
       /**
        * Index scan with different bounds.
        */
       @ParameterizedTest
       @MethodSource("indexScanParameters")
   ```



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239908977


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,961 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RW");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RO");
+
+    private static final IgniteTypeFactory TYPE_FACTORY = Commons.typeFactory();
+
+    @Mock(lenient = true)
+    private InternalTable internalTable;
+
+    @Mock(lenient = true)
+    private ExecutionContext<Object[]> ctx;
+
+    @Mock
+    private BinaryRow binaryRow;
+
+    /**
+     * Table scan.
+     */
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTableScan(boolean ro) {
+        NoOpTransaction tx = ro ? RO_TX : RW_TX;

Review Comment:
   ```suggestion
       @MethodSource("transactions")
       public void testTableScan(Transaction tx) {
   ```



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239314550


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java:
##########
@@ -63,6 +72,113 @@ public <RowT> Publisher<RowT> scan(ExecutionContext<RowT> ctx, PartitionWithTerm
             pub = internalTable.scan(partWithTerm.partId(), txAttributes.id(), recipient, null, null, null, 0, null);
         }
 
-        return StorageScanNode.convertPublisher(pub, (item) -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
+        return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
     }
+
+    /** {@inheritDoc} */
+    @Override
+    public <RowT> Publisher<RowT> indexRangeScan(ExecutionContext<RowT> ctx, PartitionWithTerm partWithTerm, RowFactory<RowT> rowFactory,
+            int indexId, String indexName, List<String> columns, @Nullable RangeCondition<RowT> cond, @Nullable BitSet requiredColumns) {

Review Comment:
   @AMashenkov Replaced ItSecondaryIndexTest tests that with unit tests for `ExpressionFactoryImplTest` that validate nulls / null bounds.



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistrySelfTest.java:
##########
@@ -79,9 +88,10 @@ public void testGetTable() {
         CompletableFuture<ExecutableTable> f = tester.getTable(tableId);
         ExecutableTable executableTable = f.join();
 
-        assertNotNull(executableTable.scanableTable());
+        assertNotNull(executableTable.scannableTable());
         assertNotNull(executableTable.updatableTable());
-        assertNotNull(executableTable.rowConverter());
+
+        verify(callback).onTableLoaded(eq(executableTable), eq(TABLE_NAME), eq(descriptor));

Review Comment:
   Done. This was 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.

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

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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: [WIP] IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1238263250


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/IndexScanNodeTest.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.stream.Collectors;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.index.ColumnCollation;
+import org.apache.ignite.internal.index.HashIndex;
+import org.apache.ignite.internal.index.IndexDescriptor;
+import org.apache.ignite.internal.index.SortedIndexDescriptor;
+import org.apache.ignite.internal.index.SortedIndexImpl;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.IgniteIndex;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+/**
+ * Test {@link IndexScanNode} execution.
+ */
+public class IndexScanNodeTest extends AbstractExecutionTest {
+
+    /**
+     * Sorted index scan execution.
+     */
+    @Test
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void testSortedIndex() {
+        IgniteIndex index = newSortedIndex();
+
+        TestScannableTable<Object[]> scannableTable = new TestScannableTable<>();
+        scannableTable.setPartitionData(0, new Object[]{4}, new Object[]{5});
+        scannableTable.setPartitionData(2, new Object[]{1}, new Object[]{2});
+
+        Comparator<Object[]> cmp = (a, b) -> {
+            Comparable o1 = (Comparable<?>) a[0];
+            Comparable o2 = (Comparable<?>) b[0];
+            return o1.compareTo(o2);
+        };

Review Comment:
   ```suggestion
           Comparator<Object[]> cmp = Comparator.comparing(arr -> (Comparable<Object>)arr[0])
   ```



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236703915


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TransformingPublisher.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.sql.engine.exec;
+
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.function.Function;
+
+/**
+ * Publisher that converts items of type {@code T} to {@code type R}.
+ */
+final class TransformingPublisher<T, R> implements Publisher<R> {
+
+    private final Publisher<T> publisher;
+
+    private final Function<T, R> function;
+
+    TransformingPublisher(Publisher<T> publisher, Function<T, R> function) {
+        this.publisher = publisher;
+        this.function = function;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void subscribe(Subscriber<? super R> subscriber) {
+        this.publisher.subscribe(new Subscriber<T>() {

Review Comment:
   Sure. Done.



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1242111123


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,952 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RO");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RW");
+
+    private static final IgniteTypeFactory TYPE_FACTORY = Commons.typeFactory();
+
+    @Mock(lenient = true)
+    private InternalTable internalTable;
+
+    @Mock(lenient = true)
+    private ExecutionContext<Object[]> ctx;
+
+    @Mock
+    private BinaryRow binaryRow;
+
+    /**
+     * Table scan.
+     */
+    @ParameterizedTest
+    @MethodSource("transactions")
+    public void testTableScan(NoOpTransaction tx) {
+        TestInput data = new TestInput();
+        data.addRow(binaryRow);
+
+        Tester tester = new Tester(data);
+
+        int partitionId = 1;
+        long term = 2;
+
+        ResultCollector collector = tester.tableScan(partitionId, term, tx);
+
+        if (tx.isReadOnly()) {
+            HybridTimestamp timestamp = tx.readTimestamp();
+            ClusterNode clusterNode = tx.clusterNode();
+
+            verify(internalTable).scan(partitionId, timestamp, clusterNode);
+        } else {
+            ClusterNode clusterNode = tx.clusterNode();
+
+            verify(internalTable).scan(partitionId, tx.id(), new PrimaryReplica(clusterNode, term), null, null, null, 0, null);
+        }
+
+        data.sendRows();
+        data.done();
+
+        collector.expectRow(binaryRow);
+        collector.expectCompleted();
+    }
+
+    /**
+     * Table scan error propagation.
+     */
+    @ParameterizedTest
+    @MethodSource("transactions")
+    public void testTableScanError(NoOpTransaction tx) {
+        TestInput input = new TestInput();
+        input.addRow(binaryRow);
+
+        Tester tester = new Tester(input);
+
+        int partitionId = 1;
+        long term = 2;
+
+        ResultCollector collector = tester.tableScan(partitionId, term, tx);
+
+        input.sendRows();
+
+        RuntimeException err = new RuntimeException("err");
+        input.sendError(err);
+
+        collector.expectRow(binaryRow);
+        collector.expectError(err);
+    }
+
+    /**
+     * Index scan with different bounds.
+     */
+    @ParameterizedTest
+    @CsvSource({
+            // RO
+            "true, INCLUSIVE, NONE",
+            "true, EXCLUSIVE, NONE",
+            "true, NONE, INCLUSIVE",
+            "true, NONE, EXCLUSIVE",
+            "true, INCLUSIVE, INCLUSIVE",
+            "true, INCLUSIVE, EXCLUSIVE",
+            "true, EXCLUSIVE, INCLUSIVE",
+            "true, EXCLUSIVE, EXCLUSIVE",
+            // RW
+            "false, INCLUSIVE, NONE",
+            "false, EXCLUSIVE, NONE",
+            "false, NONE, INCLUSIVE",
+            "false, NONE, EXCLUSIVE",
+            "false, INCLUSIVE, INCLUSIVE",
+            "false, INCLUSIVE, EXCLUSIVE",
+            "false, EXCLUSIVE, INCLUSIVE",
+            "false, EXCLUSIVE, EXCLUSIVE",
+    })

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.

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

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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: [WIP] IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1238450494


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/IndexScanNodeTest.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.stream.Collectors;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.index.ColumnCollation;
+import org.apache.ignite.internal.index.HashIndex;
+import org.apache.ignite.internal.index.IndexDescriptor;
+import org.apache.ignite.internal.index.SortedIndexDescriptor;
+import org.apache.ignite.internal.index.SortedIndexImpl;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.IgniteIndex;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+/**
+ * Test {@link IndexScanNode} execution.
+ */
+public class IndexScanNodeTest extends AbstractExecutionTest {
+
+    /**
+     * Sorted index scan execution.
+     */
+    @Test
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void testSortedIndex() {
+        IgniteIndex index = newSortedIndex();
+
+        TestScannableTable<Object[]> scannableTable = new TestScannableTable<>();
+        scannableTable.setPartitionData(0, new Object[]{4}, new Object[]{5});
+        scannableTable.setPartitionData(2, new Object[]{1}, new Object[]{2});
+
+        Comparator<Object[]> cmp = (a, b) -> {
+            Comparable o1 = (Comparable<?>) a[0];
+            Comparable o2 = (Comparable<?>) b[0];
+            return o1.compareTo(o2);
+        };
+
+        Tester tester = new Tester();
+
+        IndexScanNode<Object[]> node = tester.createSortedIndex(index, scannableTable, cmp);
+        List<Object[]> result = tester.execute(node);
+
+        tester.expectResult(result, new Object[]{1}, new Object[]{2}, new Object[]{4}, new Object[]{5});
+    }
+
+    /**
+     * Hash index lookup execution.
+     */
+    @Test
+    public void testHashIndex() {
+        IgniteIndex index = newHashIndex();
+
+        TestScannableTable<Object[]> scannableTable = new TestScannableTable<>();
+        scannableTable.setPartitionData(0, new Object[]{2}, new Object[]{1});
+        scannableTable.setPartitionData(2, new Object[]{0});
+
+        Tester tester = new Tester();
+
+        IndexScanNode<Object[]> node = tester.createHashIndex(index, scannableTable);
+        List<Object[]> result = tester.execute(node);
+
+        tester.expectResult(result, new Object[]{2}, new Object[]{1}, new Object[]{0});
+    }
+
+    private static IgniteIndex newHashIndex() {
+        IndexDescriptor descriptor = new IndexDescriptor("IDX", List.of("C1"));
+        HashIndex index = new HashIndex(1, Mockito.mock(InternalTable.class), descriptor);
+
+        return new IgniteIndex(index);
+    }
+
+    private static IgniteIndex newSortedIndex() {
+        List<String> columnNames = List.of("C1");
+        List<ColumnCollation> columnCollations = List.of(ColumnCollation.ASC_NULLS_LAST);
+        SortedIndexDescriptor descriptor = new SortedIndexDescriptor("IDX", columnNames, columnCollations);
+        SortedIndexImpl index = new SortedIndexImpl(1, Mockito.mock(InternalTable.class), descriptor);
+
+        return new IgniteIndex(index);
+    }
+
+    private class Tester {
+
+        private final ExecutionContext<Object[]> ctx;
+
+        Tester() {
+            this.ctx = executionContext();
+        }
+
+        IndexScanNode<Object[]> createSortedIndex(IgniteIndex index, TestScannableTable<?> scannableTable, Comparator<Object[]> cmp) {
+            return createIndexNode(ctx, index, scannableTable, cmp);
+        }
+
+        IndexScanNode<Object[]> createHashIndex(IgniteIndex index, TestScannableTable<?> scannableTable) {
+            return createIndexNode(ctx, index, scannableTable, null);
+        }
+
+        List<Object[]> execute(IndexScanNode<Object[]> indexNode) {
+            RootNode<Object[]> root = new RootNode<>(ctx);
+
+            root.register(indexNode);
+
+            List<Object[]> actual = new ArrayList<>();
+            while (root.hasNext()) {
+                Object[] row = root.next();
+                actual.add(row);
+            }
+
+            root.close();
+
+            return actual;
+        }
+
+        void expectResult(List<Object[]> actual, Object[]... expected) {

Review Comment:
   Fixed.



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/IndexScanNodeTest.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.stream.Collectors;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.index.ColumnCollation;
+import org.apache.ignite.internal.index.HashIndex;
+import org.apache.ignite.internal.index.IndexDescriptor;
+import org.apache.ignite.internal.index.SortedIndexDescriptor;
+import org.apache.ignite.internal.index.SortedIndexImpl;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.IgniteIndex;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+/**
+ * Test {@link IndexScanNode} execution.
+ */
+public class IndexScanNodeTest extends AbstractExecutionTest {
+
+    /**
+     * Sorted index scan execution.
+     */
+    @Test
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public void testSortedIndex() {
+        IgniteIndex index = newSortedIndex();
+
+        TestScannableTable<Object[]> scannableTable = new TestScannableTable<>();
+        scannableTable.setPartitionData(0, new Object[]{4}, new Object[]{5});
+        scannableTable.setPartitionData(2, new Object[]{1}, new Object[]{2});
+
+        Comparator<Object[]> cmp = (a, b) -> {
+            Comparable o1 = (Comparable<?>) a[0];
+            Comparable o2 = (Comparable<?>) b[0];
+            return o1.compareTo(o2);
+        };

Review Comment:
   Fixed.



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/PrimaryReplica.java:
##########
@@ -57,4 +59,26 @@ public ClusterNode node() {
     public long term() {
         return term;
     }
+
+    @Override

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.

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

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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239314685


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java:
##########
@@ -293,6 +297,10 @@ public synchronized void start() {
         services.forEach(LifecycleAware::start);
     }
 
+    public ExecutionDependencyResolver dependencyResolver() {

Review Comment:
   Removed.l



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableCallback.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.sql.engine.exec;
+
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+
+/**
+ * Callback that is called when {@link ExecutableTable} is loaded.
+ */
+@FunctionalInterface
+public interface ExecutableTableCallback {

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.

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

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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236694204


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TransformingPublisher.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.sql.engine.exec;
+
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.function.Function;
+
+/**
+ * Publisher that converts items of type {@code T} to {@code type R}.
+ */
+final class TransformingPublisher<T, R> implements Publisher<R> {

Review Comment:
   Let's move this to ignite-core module package `org.apache.ignite.internal.util.subscription`



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239901297


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,961 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RW");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RO");

Review Comment:
   ```suggestion
       private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RO");
   
       private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RW");
   ```



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239931055


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,961 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RW");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RO");

Review Comment:
   Done.



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


[GitHub] [ignite-3] AMashenkov merged pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov merged PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199


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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236697380


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TransformingPublisher.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.sql.engine.exec;
+
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.function.Function;
+
+/**
+ * Publisher that converts items of type {@code T} to {@code type R}.
+ */
+final class TransformingPublisher<T, R> implements Publisher<R> {
+
+    private final Publisher<T> publisher;
+
+    private final Function<T, R> function;
+
+    TransformingPublisher(Publisher<T> publisher, Function<T, R> function) {
+        this.publisher = publisher;
+        this.function = function;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void subscribe(Subscriber<? super R> subscriber) {
+        this.publisher.subscribe(new Subscriber<T>() {

Review Comment:
   Let's make anonymous Subscriber a static nested class or a top-level class.



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236719996


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java:
##########
@@ -63,6 +72,113 @@ public <RowT> Publisher<RowT> scan(ExecutionContext<RowT> ctx, PartitionWithTerm
             pub = internalTable.scan(partWithTerm.partId(), txAttributes.id(), recipient, null, null, null, 0, null);
         }
 
-        return StorageScanNode.convertPublisher(pub, (item) -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
+        return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
     }
+
+    /** {@inheritDoc} */
+    @Override
+    public <RowT> Publisher<RowT> indexRangeScan(ExecutionContext<RowT> ctx, PartitionWithTerm partWithTerm, RowFactory<RowT> rowFactory,
+            int indexId, String indexName, List<String> columns, @Nullable RangeCondition<RowT> cond, @Nullable BitSet requiredColumns) {
+
+        BinaryTupleSchema indexRowSchema = RowConverter.createIndexRowSchema(columns, tableDescriptor);
+        TxAttributes txAttributes = ctx.txAttributes();
+
+        Publisher<BinaryRow> pub;
+        BinaryTuplePrefix lower;
+        BinaryTuplePrefix upper;
+
+        int flags = 0;
+
+        if (cond == null) {
+            flags = SortedIndex.INCLUDE_LEFT | SortedIndex.INCLUDE_RIGHT;
+            lower = null;
+            upper = null;
+        } else {
+            lower = toBinaryTuplePrefix(ctx, indexRowSchema, cond.lower(), rowFactory);
+            upper = toBinaryTuplePrefix(ctx, indexRowSchema, cond.upper(), rowFactory);
+
+            flags |= (cond.lowerInclude()) ? SortedIndex.INCLUDE_LEFT : 0;
+            flags |= (cond.upperInclude()) ? SortedIndex.INCLUDE_RIGHT : 0;
+        }
+
+        if (txAttributes.readOnly()) {
+            pub = internalTable.scan(
+                    partWithTerm.partId(),
+                    txAttributes.time(),
+                    ctx.localNode(),
+                    indexId,
+                    lower,
+                    upper,
+                    flags,
+                    requiredColumns
+            );
+        } else {
+            pub = internalTable.scan(
+                    partWithTerm.partId(),
+                    txAttributes.id(),
+                    new PrimaryReplica(ctx.localNode(), partWithTerm.term()),
+                    indexId,
+                    lower,
+                    upper,
+                    flags,
+                    requiredColumns
+            );
+        }
+
+        return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
+    }
+
+    @Override
+    public <RowT> Publisher<RowT> indexLookup(ExecutionContext<RowT> ctx, PartitionWithTerm partWithTerm, RowFactory<RowT> rowFactory,
+            int indexId, String indexName, List<String> columns, RowT key, @Nullable BitSet requiredColumns) {

Review Comment:
   ```suggestion
       public <RowT> Publisher<RowT> indexLookup(
               ExecutionContext<RowT> ctx,
               PartitionWithTerm partWithTerm, 
               RowFactory<RowT> rowFactory,
               int indexId, 
               List<String> columns,
               RowT key, 
               @Nullable BitSet requiredColumns
   ) {
   ```



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236762824


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java:
##########
@@ -63,6 +72,113 @@ public <RowT> Publisher<RowT> scan(ExecutionContext<RowT> ctx, PartitionWithTerm
             pub = internalTable.scan(partWithTerm.partId(), txAttributes.id(), recipient, null, null, null, 0, null);
         }
 
-        return StorageScanNode.convertPublisher(pub, (item) -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
+        return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
     }
+
+    /** {@inheritDoc} */
+    @Override
+    public <RowT> Publisher<RowT> indexRangeScan(ExecutionContext<RowT> ctx, PartitionWithTerm partWithTerm, RowFactory<RowT> rowFactory,
+            int indexId, String indexName, List<String> columns, @Nullable RangeCondition<RowT> cond, @Nullable BitSet requiredColumns) {
+
+        BinaryTupleSchema indexRowSchema = RowConverter.createIndexRowSchema(columns, tableDescriptor);
+        TxAttributes txAttributes = ctx.txAttributes();
+
+        Publisher<BinaryRow> pub;
+        BinaryTuplePrefix lower;
+        BinaryTuplePrefix upper;
+
+        int flags = 0;
+
+        if (cond == null) {
+            flags = SortedIndex.INCLUDE_LEFT | SortedIndex.INCLUDE_RIGHT;
+            lower = null;
+            upper = null;
+        } else {
+            lower = toBinaryTuplePrefix(ctx, indexRowSchema, cond.lower(), rowFactory);
+            upper = toBinaryTuplePrefix(ctx, indexRowSchema, cond.upper(), rowFactory);
+
+            flags |= (cond.lowerInclude()) ? SortedIndex.INCLUDE_LEFT : 0;
+            flags |= (cond.upperInclude()) ? SortedIndex.INCLUDE_RIGHT : 0;
+        }
+
+        if (txAttributes.readOnly()) {
+            pub = internalTable.scan(
+                    partWithTerm.partId(),
+                    txAttributes.time(),
+                    ctx.localNode(),
+                    indexId,
+                    lower,
+                    upper,
+                    flags,
+                    requiredColumns
+            );
+        } else {
+            pub = internalTable.scan(
+                    partWithTerm.partId(),
+                    txAttributes.id(),
+                    new PrimaryReplica(ctx.localNode(), partWithTerm.term()),
+                    indexId,
+                    lower,
+                    upper,
+                    flags,
+                    requiredColumns
+            );
+        }
+
+        return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
+    }
+
+    @Override
+    public <RowT> Publisher<RowT> indexLookup(ExecutionContext<RowT> ctx, PartitionWithTerm partWithTerm, RowFactory<RowT> rowFactory,
+            int indexId, String indexName, List<String> columns, RowT key, @Nullable BitSet requiredColumns) {

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.

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

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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236760687


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java:
##########
@@ -63,6 +72,113 @@ public <RowT> Publisher<RowT> scan(ExecutionContext<RowT> ctx, PartitionWithTerm
             pub = internalTable.scan(partWithTerm.partId(), txAttributes.id(), recipient, null, null, null, 0, null);
         }
 
-        return StorageScanNode.convertPublisher(pub, (item) -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
+        return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
     }
+
+    /** {@inheritDoc} */
+    @Override
+    public <RowT> Publisher<RowT> indexRangeScan(ExecutionContext<RowT> ctx, PartitionWithTerm partWithTerm, RowFactory<RowT> rowFactory,
+            int indexId, String indexName, List<String> columns, @Nullable RangeCondition<RowT> cond, @Nullable BitSet requiredColumns) {

Review Comment:
   @AMashenkov See  runner / `ItSecondaryIndexTest testNullsInSearchRow`.



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236709047


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TransformingPublisher.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.sql.engine.exec;
+
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.function.Function;
+
+/**
+ * Publisher that converts items of type {@code T} to {@code type R}.
+ */
+final class TransformingPublisher<T, R> implements Publisher<R> {
+
+    private final Publisher<T> publisher;
+
+    private final Function<T, R> function;
+
+    TransformingPublisher(Publisher<T> publisher, Function<T, R> function) {
+        this.publisher = publisher;
+        this.function = function;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void subscribe(Subscriber<? super R> subscriber) {
+        this.publisher.subscribe(new Subscriber<T>() {

Review Comment:
   Well, I missed your first comment. I am going to move TransformingPublisher to `org.apache.ignite.internal.util.subscription` and add some unit tests as well.



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236759236


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java:
##########
@@ -63,6 +72,113 @@ public <RowT> Publisher<RowT> scan(ExecutionContext<RowT> ctx, PartitionWithTerm
             pub = internalTable.scan(partWithTerm.partId(), txAttributes.id(), recipient, null, null, null, 0, null);
         }
 
-        return StorageScanNode.convertPublisher(pub, (item) -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
+        return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
     }
+
+    /** {@inheritDoc} */
+    @Override
+    public <RowT> Publisher<RowT> indexRangeScan(ExecutionContext<RowT> ctx, PartitionWithTerm partWithTerm, RowFactory<RowT> rowFactory,
+            int indexId, String indexName, List<String> columns, @Nullable RangeCondition<RowT> cond, @Nullable BitSet requiredColumns) {

Review Comment:
   @AMashenkov I added indexName to simplify testing  because there is no way to connect `indexId` and a call index scan/lookup methods (yes, there are tests in the main branch that intercept index scans and we are going to need this anyways)). 
   An alternative would require moving index scans/lookup to a separate class + making all the necessary changes to make it accessible from `LogicalRelImplementor::visit(IgniteIndexScan rel)`. I think this approach would create a lot of unnecessary complexity.
   
   



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236763358


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TransformingPublisher.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.sql.engine.exec;
+
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.function.Function;
+
+/**
+ * Publisher that converts items of type {@code T} to {@code type R}.
+ */
+final class TransformingPublisher<T, R> implements Publisher<R> {
+
+    private final Publisher<T> publisher;
+
+    private final Function<T, R> function;
+
+    TransformingPublisher(Publisher<T> publisher, Function<T, R> function) {
+        this.publisher = publisher;
+        this.function = function;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void subscribe(Subscriber<? super R> subscriber) {
+        this.publisher.subscribe(new Subscriber<T>() {

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.

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

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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239928045


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,961 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RW");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RO");
+
+    private static final IgniteTypeFactory TYPE_FACTORY = Commons.typeFactory();
+
+    @Mock(lenient = true)
+    private InternalTable internalTable;
+
+    @Mock(lenient = true)
+    private ExecutionContext<Object[]> ctx;
+
+    @Mock
+    private BinaryRow binaryRow;
+
+    /**
+     * Table scan.
+     */
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTableScan(boolean ro) {
+        NoOpTransaction tx = ro ? RO_TX : RW_TX;
+
+        TestInput data = new TestInput();
+        data.addRow(binaryRow);
+
+        Tester tester = new Tester(data);
+
+        int partitionId = 1;
+        long term = 2;
+
+        ResultCollector collector = tester.tableScan(partitionId, term, tx);
+
+        if (ro) {

Review Comment:
   @AMashenkov 
   Thanks. Applied this every where expect for  `testIndexScan(boolean ro, Bound lower, Bound upper)`
   



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,961 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RW");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RO");
+
+    private static final IgniteTypeFactory TYPE_FACTORY = Commons.typeFactory();
+
+    @Mock(lenient = true)
+    private InternalTable internalTable;
+
+    @Mock(lenient = true)
+    private ExecutionContext<Object[]> ctx;
+
+    @Mock
+    private BinaryRow binaryRow;
+
+    /**
+     * Table scan.
+     */
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTableScan(boolean ro) {
+        NoOpTransaction tx = ro ? RO_TX : RW_TX;

Review Comment:
   Done.



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1242013058


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,952 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RO");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RW");
+
+    private static final IgniteTypeFactory TYPE_FACTORY = Commons.typeFactory();
+
+    @Mock(lenient = true)
+    private InternalTable internalTable;
+
+    @Mock(lenient = true)
+    private ExecutionContext<Object[]> ctx;
+
+    @Mock
+    private BinaryRow binaryRow;
+
+    /**
+     * Table scan.
+     */
+    @ParameterizedTest
+    @MethodSource("transactions")
+    public void testTableScan(NoOpTransaction tx) {
+        TestInput data = new TestInput();
+        data.addRow(binaryRow);
+
+        Tester tester = new Tester(data);
+
+        int partitionId = 1;
+        long term = 2;
+
+        ResultCollector collector = tester.tableScan(partitionId, term, tx);
+
+        if (tx.isReadOnly()) {
+            HybridTimestamp timestamp = tx.readTimestamp();
+            ClusterNode clusterNode = tx.clusterNode();
+
+            verify(internalTable).scan(partitionId, timestamp, clusterNode);
+        } else {
+            ClusterNode clusterNode = tx.clusterNode();
+
+            verify(internalTable).scan(partitionId, tx.id(), new PrimaryReplica(clusterNode, term), null, null, null, 0, null);
+        }
+
+        data.sendRows();
+        data.done();
+
+        collector.expectRow(binaryRow);
+        collector.expectCompleted();
+    }
+
+    /**
+     * Table scan error propagation.
+     */
+    @ParameterizedTest
+    @MethodSource("transactions")
+    public void testTableScanError(NoOpTransaction tx) {
+        TestInput input = new TestInput();
+        input.addRow(binaryRow);
+
+        Tester tester = new Tester(input);
+
+        int partitionId = 1;
+        long term = 2;
+
+        ResultCollector collector = tester.tableScan(partitionId, term, tx);
+
+        input.sendRows();
+
+        RuntimeException err = new RuntimeException("err");
+        input.sendError(err);
+
+        collector.expectRow(binaryRow);
+        collector.expectError(err);
+    }
+
+    /**
+     * Index scan with different bounds.
+     */
+    @ParameterizedTest
+    @CsvSource({
+            // RO
+            "true, INCLUSIVE, NONE",
+            "true, EXCLUSIVE, NONE",
+            "true, NONE, INCLUSIVE",
+            "true, NONE, EXCLUSIVE",
+            "true, INCLUSIVE, INCLUSIVE",
+            "true, INCLUSIVE, EXCLUSIVE",
+            "true, EXCLUSIVE, INCLUSIVE",
+            "true, EXCLUSIVE, EXCLUSIVE",
+            // RW
+            "false, INCLUSIVE, NONE",
+            "false, EXCLUSIVE, NONE",
+            "false, NONE, INCLUSIVE",
+            "false, NONE, EXCLUSIVE",
+            "false, INCLUSIVE, INCLUSIVE",
+            "false, INCLUSIVE, EXCLUSIVE",
+            "false, EXCLUSIVE, INCLUSIVE",
+            "false, EXCLUSIVE, EXCLUSIVE",
+    })

Review Comment:
   ```suggestion
       static Stream<Arguments> indexScanParameters() {
           List<Arguments> args = new ArrayList<>();
           
           for(Bound leftBound : Bound.values) {
               for(Bound rightBound : Bound.values) {
                   arguments.add(Argtments.of(NoOpTransaction.readOnly("RO"), leftBound, rightBound));
                   arguments.add(Argtments.of(NoOpTransaction.readWrite("RW"), leftBound, rightBound));
               }
               
               return args.stream();
           }
       }
       
       /**
        * Index scan with different bounds.
        */
       @ParameterizedTest
       @MethodSource({
               // RO
               "true, INCLUSIVE, NONE",
               "true, EXCLUSIVE, NONE",
               "true, NONE, INCLUSIVE",
               "true, NONE, EXCLUSIVE",
               "true, INCLUSIVE, INCLUSIVE",
               "true, INCLUSIVE, EXCLUSIVE",
               "true, EXCLUSIVE, INCLUSIVE",
               "true, EXCLUSIVE, EXCLUSIVE",
               // RW
               "false, INCLUSIVE, NONE",
               "false, EXCLUSIVE, NONE",
               "false, NONE, INCLUSIVE",
               "false, NONE, EXCLUSIVE",
               "false, INCLUSIVE, INCLUSIVE",
               "false, INCLUSIVE, EXCLUSIVE",
               "false, EXCLUSIVE, INCLUSIVE",
               "false, EXCLUSIVE, EXCLUSIVE",
       })
   ```



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236860441


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TransformingPublisher.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.sql.engine.exec;
+
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.function.Function;
+
+/**
+ * Publisher that converts items of type {@code T} to {@code type R}.
+ */
+final class TransformingPublisher<T, R> implements Publisher<R> {

Review Comment:
   Done.



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1236712245


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java:
##########
@@ -63,6 +72,113 @@ public <RowT> Publisher<RowT> scan(ExecutionContext<RowT> ctx, PartitionWithTerm
             pub = internalTable.scan(partWithTerm.partId(), txAttributes.id(), recipient, null, null, null, 0, null);
         }
 
-        return StorageScanNode.convertPublisher(pub, (item) -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
+        return new TransformingPublisher<>(pub, item -> rowConverter.toRow(ctx, item, rowFactory, requiredColumns));
     }
+
+    /** {@inheritDoc} */
+    @Override
+    public <RowT> Publisher<RowT> indexRangeScan(ExecutionContext<RowT> ctx, PartitionWithTerm partWithTerm, RowFactory<RowT> rowFactory,
+            int indexId, String indexName, List<String> columns, @Nullable RangeCondition<RowT> cond, @Nullable BitSet requiredColumns) {

Review Comment:
   ```suggestion
       public <RowT> Publisher<RowT> indexRangeScan(
               ExecutionContext<RowT> ctx, 
               PartitionWithTerm partWithTerm,
               RowFactory<RowT> rowFactory,
               int indexId, 
               String indexName, 
               List<String> columns, 
               @Nullable RangeCondition<RowT> cond, 
               @Nullable BitSet requiredColumns
       ) {
   ```



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


[GitHub] [ignite-3] lowka commented on a diff in pull request #2199: [WIP] IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "lowka (via GitHub)" <gi...@apache.org>.
lowka commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1238450830


##########
modules/table/src/main/java/org/apache/ignite/internal/utils/PrimaryReplica.java:
##########
@@ -57,4 +59,26 @@ public ClusterNode node() {
     public long term() {
         return term;
     }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        PrimaryReplica that = (PrimaryReplica) o;
+        return term == that.term && Objects.equals(node, that.node);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(node, term);
+    }
+
+    @Override

Review Comment:
   Fixed.



##########
modules/table/src/main/java/org/apache/ignite/internal/utils/PrimaryReplica.java:
##########
@@ -57,4 +59,26 @@ public ClusterNode node() {
     public long term() {
         return term;
     }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        PrimaryReplica that = (PrimaryReplica) o;
+        return term == that.term && Objects.equals(node, that.node);
+    }
+
+    @Override

Review Comment:
   Fixed.



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/NoOpTransaction.java:
##########
@@ -43,14 +43,32 @@ public final class NoOpTransaction implements InternalTransaction {
 
     private final TablePartitionId groupId = new TablePartitionId(1, 0);
 
+    private final boolean readOnly;
+
     /**
-     * Constructs the object.
+     * Constructs a read only transaction.
      *
      * @param name Name of the node.
      */
     public NoOpTransaction(String name) {
+        this(name, true);
+    }
+
+    /**
+     * Constructs a transaction.
+     *
+     * @param name Name of the node.
+     * @param readOnly Read-only or not.
+     */
+    public NoOpTransaction(String name, boolean readOnly) {

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.

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

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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239909194


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,961 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RW");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RO");
+
+    private static final IgniteTypeFactory TYPE_FACTORY = Commons.typeFactory();
+
+    @Mock(lenient = true)
+    private InternalTable internalTable;
+
+    @Mock(lenient = true)
+    private ExecutionContext<Object[]> ctx;
+
+    @Mock
+    private BinaryRow binaryRow;
+
+    /**
+     * Table scan.
+     */
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testTableScan(boolean ro) {
+        NoOpTransaction tx = ro ? RO_TX : RW_TX;
+
+        TestInput data = new TestInput();
+        data.addRow(binaryRow);
+
+        Tester tester = new Tester(data);
+
+        int partitionId = 1;
+        long term = 2;
+
+        ResultCollector collector = tester.tableScan(partitionId, term, tx);
+
+        if (ro) {

Review Comment:
   ```suggestion
           if (tx.isReadOnly()) {
   ```



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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2199: IGNITE-19726: Sql. Migrate index operations to ScannableTable.

Posted by "AMashenkov (via GitHub)" <gi...@apache.org>.
AMashenkov commented on code in PR #2199:
URL: https://github.com/apache/ignite-3/pull/2199#discussion_r1239908411


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ScannableTableSelfTest.java:
##########
@@ -0,0 +1,961 @@
+/*
+ * 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.sql.engine.exec.rel;
+
+import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryImpl.UNSPECIFIED_VALUE_PLACEHOLDER;
+import static org.apache.ignite.lang.IgniteStringFormatter.format;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.ArgumentMatchers.isNull;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+import java.util.concurrent.SubmissionPublisher;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.IntStream;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.index.SortedIndex;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTuplePrefix;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTable;
+import org.apache.ignite.internal.sql.engine.exec.ScannableTableImpl;
+import org.apache.ignite.internal.sql.engine.exec.TableRowConverter;
+import org.apache.ignite.internal.sql.engine.exec.TxAttributes;
+import org.apache.ignite.internal.sql.engine.exec.exp.RangeCondition;
+import org.apache.ignite.internal.sql.engine.framework.NoOpTransaction;
+import org.apache.ignite.internal.sql.engine.metadata.PartitionWithTerm;
+import org.apache.ignite.internal.sql.engine.planner.AbstractPlannerTest.TestTableDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.table.InternalTable;
+import org.apache.ignite.internal.utils.PrimaryReplica;
+import org.apache.ignite.network.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Unit tests for {@link ScannableTableImpl}. We check that scan/lookup operations call appropriate methods
+ * of the underlying APIs with required arguments.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ScannableTableSelfTest {
+
+    private static final NoOpTransaction RO_TX = NoOpTransaction.readOnly("RW");
+
+    private static final NoOpTransaction RW_TX = NoOpTransaction.readWrite("RO");

Review Comment:
   ```suggestion
        static Stream<Arguments> transactions() {
           return Stream.of(
                   Arguments.of(Named.of("Read-only transaction", NoOpTransaction.readOnly("RO"))),
                   Arguments.of(Named.of("Read-write transaction", NoOpTransaction.readWrite("RW"))),
           );
       }
   ```



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