This is an automated email from the ASF dual-hosted git repository.

amashenkov pushed a commit to branch ignite-21732
in repository https://gitbox.apache.org/repos/asf/ignite-3.git

commit 552f8ac44fe9eef23b4cc520912c8421fe775536
Author: amashenkov <andrey.mashen...@gmail.com>
AuthorDate: Tue Apr 30 13:41:11 2024 +0300

    Add test.
---
 .../exec/ProjectedTableRowConverterImpl.java       |   6 +-
 .../sql/engine/exec/TableRowConverterImpl.java     |  10 +-
 .../exec/ProjectedTableRowConverterSelfTest.java   | 122 +++++++++++++++++++++
 3 files changed, 129 insertions(+), 9 deletions(-)

diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterImpl.java
index a136b9de3c..d7ece09da3 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterImpl.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.schema.BinaryTupleSchema;
 import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.schema.SchemaRegistry;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
 import 
org.apache.ignite.internal.sql.engine.util.FieldDeserializingProjectedTuple;
 import org.apache.ignite.internal.sql.engine.util.FormatAwareProjectedTuple;
 
@@ -63,7 +64,7 @@ public class ProjectedTableRowConverterImpl extends 
TableRowConverterImpl {
     }
 
     @Override
-    protected InternalTuple asInternalTuple(BinaryRow tableRow) {
+    public <RowT> RowT toRow(ExecutionContext<RowT> ectx, BinaryRow tableRow, 
RowFactory<RowT> factory) {
         InternalTuple tuple;
         if (tableRow.schemaVersion() == schemaDescriptor.version()) {
             BinaryTuple tableTuple = new 
BinaryTuple(schemaDescriptor.length(), tableRow.tupleSlice());
@@ -78,6 +79,7 @@ public class ProjectedTableRowConverterImpl extends 
TableRowConverterImpl {
                     requiredColumnsMapping
             );
         }
-        return tuple;
+
+        return factory.create(tuple);
     }
 }
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterImpl.java
index c2d7178f8b..b3e6bb98ce 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterImpl.java
@@ -57,18 +57,14 @@ public class TableRowConverterImpl implements 
TableRowConverter {
             BinaryRow tableRow,
             RowHandler.RowFactory<RowT> factory
     ) {
-        InternalTuple tuple = asInternalTuple(tableRow);
-
-        return factory.create(tuple);
-    }
-
-    protected InternalTuple asInternalTuple(BinaryRow tableRow) {
         InternalTuple tuple;
+
         if (tableRow.schemaVersion() == schemaDescriptor.version()) {
             tuple = new BinaryTuple(schemaDescriptor.length(), 
tableRow.tupleSlice());
         } else {
             tuple= schemaRegistry.resolve(tableRow, schemaDescriptor);
         }
-        return tuple;
+
+        return factory.create(tuple);
     }
 }
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterSelfTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterSelfTest.java
new file mode 100644
index 0000000000..796ac9248a
--- /dev/null
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterSelfTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import org.apache.calcite.util.BitSets;
+import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.BinaryRowImpl;
+import org.apache.ignite.internal.schema.BinaryTupleSchema;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.SchemaRegistry;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.sql.engine.exec.SqlRowHandler.RowWrapper;
+import org.apache.ignite.internal.sql.engine.exec.row.RowSchema;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.apache.ignite.internal.type.NativeType;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Tests for {@link TableRowConverterImpl}.
+ */
+@ExtendWith(MockitoExtension.class)
+public class ProjectedTableRowConverterSelfTest extends BaseIgniteAbstractTest 
{
+
+    @Mock
+    private SchemaRegistry schemaRegistry;
+
+    @Mock
+    private ExecutionContext<RowWrapper> executionContext;
+
+    /** Test checks conversion from storage row to execution engine row. */
+    @Test
+    public void testToEngineRowSameVersion() {
+        SchemaDescriptor schema = newSchema(
+                List.of(
+                        Map.entry("c3", NativeTypes.UUID),
+                        Map.entry("c2", NativeTypes.STRING),
+                        Map.entry("c4", NativeTypes.BOOLEAN),
+                        Map.entry("c1", NativeTypes.INT32)
+                        ),
+                List.of("c1"),
+                null
+        );
+
+        RowSchema rowSchema = RowSchema.builder()
+                .addField(NativeTypes.STRING)
+                .addField(NativeTypes.INT32)
+                .build();
+
+        RowHandler<RowWrapper> rowHandler = SqlRowHandler.INSTANCE;
+        RowFactory<RowWrapper> rowFactory = rowHandler.factory(rowSchema);
+
+        ByteBuffer tupleBuf = new BinaryTupleBuilder(schema.length())
+                .appendUuid(UUID.randomUUID())
+                .appendString("ABC")
+                .appendBoolean(true)
+                .appendInt(100)
+                .build();
+
+        BinaryRow binaryRow = new BinaryRowImpl(schema.version(), tupleBuf);
+
+        ProjectedTableRowConverterImpl converter = new 
ProjectedTableRowConverterImpl(
+                schemaRegistry,
+                BinaryTupleSchema.createRowSchema(schema),
+                schema,
+                BitSets.of(1, 3)
+        );
+
+        RowWrapper row = converter.toRow(executionContext, binaryRow, 
rowFactory);
+        assertEquals("ABC", rowHandler.get(0, row));
+        assertEquals(100, rowHandler.get(1, row));
+    }
+
+    private static SchemaDescriptor newSchema(
+            List<Map.Entry<String, NativeType>> columns,
+            List<String> keyColumns,
+            @Nullable List<String> colocationColumns
+    ) {
+        List<Column> columnList = columns.stream()
+                .map(entry -> {
+                    String name = entry.getKey();
+                    NativeType type = entry.getValue();
+
+                    return new Column(name, type, !keyColumns.contains(name));
+                })
+                .collect(Collectors.toList());
+        return new SchemaDescriptor(
+                1,
+                columnList,
+                keyColumns,
+                colocationColumns
+        );
+    }
+}

Reply via email to