You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2022/02/08 15:43:28 UTC

[ignite-3] branch main updated: IGNITE-16422 Fix ArrayOutOfBoundException on row update via SQL (#606)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new d0eab12  IGNITE-16422 Fix ArrayOutOfBoundException on row update via SQL (#606)
d0eab12 is described below

commit d0eab124696e9d11c2dc5f171921859b3e90b0ba
Author: Vladimir Ermakov <85...@users.noreply.github.com>
AuthorDate: Tue Feb 8 18:43:24 2022 +0300

    IGNITE-16422 Fix ArrayOutOfBoundException on row update via SQL (#606)
---
 .../app/jdbc/ItJdbcUpdateStatementSelfTest.java    | 109 +++++++++++++++++----
 .../sql/engine/schema/IgniteTableImpl.java         |  74 +++++++++-----
 2 files changed, 138 insertions(+), 45 deletions(-)

diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcUpdateStatementSelfTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcUpdateStatementSelfTest.java
index baaa1a1..dbe6909 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcUpdateStatementSelfTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/jdbc/ItJdbcUpdateStatementSelfTest.java
@@ -18,10 +18,13 @@
 package org.apache.ignite.internal.runner.app.jdbc;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 
+import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.function.Consumer;
 import org.apache.ignite.table.KeyValueView;
-import org.apache.ignite.table.Tuple;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
@@ -29,41 +32,109 @@ import org.junit.jupiter.api.Test;
  * Update statement self test.
  */
 @Disabled("https://issues.apache.org/jira/browse/IGNITE-15655")
-public class ItJdbcUpdateStatementSelfTest extends ItJdbcAbstractStatementSelfTest {
+public class ItJdbcUpdateStatementSelfTest extends AbstractJdbcSelfTest {
     /**
      * Execute test.
      *
      * @throws SQLException If failed.
      */
     @Test
-    public void testExecute() throws SQLException {
-        stmt.execute("update PUBLIC.PERSON set firstName = 'Jack' where substring(SID, 2, 1)::int % 2 = 0");
+    public void testExecuteUpdateMultipleColumnsUpdate() throws Exception {
+        final String q1 = "CREATE TABLE usertable (\n"
+                + "    ycsb_key1 int NOT NULL,\n"
+                + "    ycsb_key2 varchar(3) NOT NULL,\n"
+                + "    field1   varchar(100),\n"
+                + "    field2   varchar(100),\n"
+                + "    field3   varchar(100),\n"
+                + "    field4   varchar(100),\n"
+                + "    field5   varchar(100),\n"
+                + "    field6   varchar(100),\n"
+                + "    field7   varchar(100),\n"
+                + "    field8   varchar(100),\n"
+                + "    field9   varchar(100),\n"
+                + "    field10  varchar(100),\n"
+                + "    PRIMARY KEY(ycsb_key1, ycsb_key2)\n"
+                + ");";
 
-        KeyValueView<Tuple, Tuple> person = clusterNodes.get(0).tables()
-                .table("PUBLIC.PERSON").keyValueView();
+        final String q2 = "INSERT INTO usertable values(1, 'key', 'a1','a2','a3','a4','a5','a6','a7','a8','a9','a10');";
 
-        assertEquals("John", person.get(null, Tuple.create().set("ID", 1)).stringValue("FIRSTNAME"));
-        assertEquals("Jack", person.get(null, Tuple.create().set("ID", 2)).stringValue("FIRSTNAME"));
-        assertEquals("Mike", person.get(null, Tuple.create().set("ID", 3)).stringValue("FIRSTNAME"));
+        final String q3 = "UPDATE usertable SET FIELD1='b1',FIELD2='b2',FIELD3='b3',FIELD4='b4',FIELD5='b5',"
+                + "FIELD6='b6',FIELD7='b7',FIELD8='b8',FIELD9='b9',FIELD10='b10' WHERE YCSB_KEY1=1";
+
+        final String q4 = "DROP TABLE usertable;";
+
+        assertEquals(0, stmt.executeUpdate(q1));
+        assertEquals(1, stmt.executeUpdate(q2));
+        assertEquals(1, stmt.executeUpdate(q3));
+
+        stmt.executeQuery("SELECT * FROM usertable WHERE YCSB_KEY1=1;");
+
+        ResultSet resultSet = stmt.getResultSet();
+
+        assertTrue(resultSet.next());
+
+        for (int i = 1; i < 11; i++) {
+            assertEquals("b" + i, resultSet.getString(i + 2));
+        }
+
+        assertEquals(0, stmt.executeUpdate(q4));
     }
 
     /**
-     * Execute update test.
+     * Execute test.
      *
      * @throws SQLException If failed.
      */
     @Test
-    public void testExecuteUpdate() throws SQLException {
-        int i = stmt.executeUpdate(
-                "update PUBLIC.PERSON set firstName = 'Jack' where substring(SID, 2, 1)::int % 2 = 0");
+    public void testExecuteAndExecuteUpdate() throws SQLException {
+        testExecute((String updateQuery) -> {
+            try {
+                stmt.executeUpdate(updateQuery);
+            } catch (SQLException e) {
+                fail(e);
+            }
+        });
 
-        assertEquals(1, i);
+        testExecute((String updateQuery) -> {
+            try {
+                stmt.execute(updateQuery);
+            } catch (SQLException e) {
+                fail(e);
+            }
+        });
+    }
 
-        KeyValueView<Tuple, Tuple> person = clusterNodes.get(0).tables()
-                .table("PUBLIC.PERSON").keyValueView();
+    private void testExecute(Consumer<String> updateFunction) throws SQLException {
+        final String createSql = "CREATE TABLE public.person(id INTEGER PRIMARY KEY, sid VARCHAR,"
+                + " firstname VARCHAR NOT NULL, lastname VARCHAR NOT NULL, age INTEGER NOT NULL)";
+
+        final String insertSql = "INSERT INTO public.person(sid, id, firstname, lastname, age) VALUES "
+                + "('p1', 1, 'John', 'White', 25), "
+                + "('p2', 2, 'Joe', 'Black', 35), "
+                + "('p3', 3, 'Mike', 'Green', 40)";
+
+        final String updateSql = "update PUBLIC.PERSON set firstName = 'Jack' where substring(SID, 2, 1)::int % 2 = 0";
+
+        final String dropSql = "DROP TABLE PUBLIC.PERSON;";
+
+        updateFunction.accept(createSql);
+        updateFunction.accept(insertSql);
+        updateFunction.accept(updateSql);
+
+        KeyValueView<Integer, Person> person = clusterNodes.get(0).tables()
+                .table("PUBLIC.PERSON").keyValueView(Integer.class, Person.class);
+
+        assertEquals("John", person.get(null, 1).firstname);
+        assertEquals("Jack", person.get(null, 2).firstname);
+        assertEquals("Mike", person.get(null, 3).firstname);
+
+        updateFunction.accept(dropSql);
+    }
 
-        assertEquals("John", person.get(null, Tuple.create().set("ID", 1)).stringValue("FIRSTNAME"));
-        assertEquals("Jack", person.get(null, Tuple.create().set("ID", 2)).stringValue("FIRSTNAME"));
-        assertEquals("Mike", person.get(null, Tuple.create().set("ID", 3)).stringValue("FIRSTNAME"));
+    private static class Person {
+        public int age;
+        public String sid;
+        public String firstname;
+        public String lastname;
     }
 }
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
index 08d0251..6443f94 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
@@ -17,13 +17,14 @@
 
 package org.apache.ignite.internal.sql.engine.schema;
 
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
+import java.util.Objects;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
@@ -303,46 +304,67 @@ public class IgniteTableImpl extends AbstractTable implements InternalIgniteTabl
     }
 
     private <RowT> BinaryRow updateTuple(RowT row, List<String> updateColList, ExecutionContext<RowT> ectx) {
+        RowHandler<RowT> hnd = ectx.rowHandler();
+
+        Object2IntMap<String> columnToIndex = new Object2IntOpenHashMap<>(updateColList.size());
+
+        for (int i = 0; i < updateColList.size(); i++) {
+            columnToIndex.put(updateColList.get(i), i + desc.columnsCount());
+        }
+
         int nonNullVarlenKeyCols = 0;
         int nonNullVarlenValCols = 0;
 
-        RowHandler<RowT> hnd = ectx.rowHandler();
-        int offset = desc.columnsCount();
-        Set<String> toUpdate = new HashSet<>(updateColList);
+        int keyOffset = schemaDescriptor.keyColumns().firstVarlengthColumn();
 
-        for (ColumnDescriptor colDesc : columnsOrderedByPhysSchema) {
-            if (colDesc.physicalType().spec().fixedLength()) {
-                continue;
-            }
+        if (keyOffset > -1) {
+            nonNullVarlenKeyCols = countNotNullColumns(
+                    keyOffset,
+                    schemaDescriptor.keyColumns().length(),
+                    columnToIndex, hnd, row);
+        }
 
-            Object val = toUpdate.contains(colDesc.name())
-                    ? hnd.get(colDesc.logicalIndex() + offset, row)
-                    : hnd.get(colDesc.logicalIndex(), row);
+        int valOffset = schemaDescriptor.valueColumns().firstVarlengthColumn();
 
-            if (val != null) {
-                if (colDesc.key()) {
-                    nonNullVarlenKeyCols++;
-                } else {
-                    nonNullVarlenValCols++;
-                }
-            }
+        if (valOffset > -1) {
+            nonNullVarlenValCols = countNotNullColumns(
+                    schemaDescriptor.keyColumns().length() + valOffset,
+                    schemaDescriptor.length(),
+                    columnToIndex, hnd, row);
         }
 
         RowAssembler rowAssembler = new RowAssembler(schemaDescriptor, nonNullVarlenKeyCols, nonNullVarlenValCols);
 
         for (ColumnDescriptor colDesc : columnsOrderedByPhysSchema) {
-            RowAssembler.writeValue(
-                    rowAssembler,
-                    colDesc.physicalType(),
-                    toUpdate.contains(colDesc.name())
-                            ? hnd.get(colDesc.logicalIndex() + offset, row)
-                            : hnd.get(colDesc.logicalIndex(), row)
-            );
+            int orDefault = columnToIndex.getOrDefault(colDesc.name(), colDesc.logicalIndex());
+
+            Object val = hnd.get(orDefault, row);
+
+            RowAssembler.writeValue(rowAssembler, colDesc.physicalType(), val);
         }
 
         return rowAssembler.build();
     }
 
+    private <RowT> int countNotNullColumns(int start, int end, Object2IntMap<String> columnToIndex,
+            RowHandler<RowT> hnd, RowT row) {
+        int nonNullCols = 0;
+
+        for (int i = start; i < end; i++) {
+            ColumnDescriptor colDesc = Objects.requireNonNull(columnsOrderedByPhysSchema.get(i));
+
+            assert !colDesc.physicalType().spec().fixedLength();
+
+            int colIdInRow = columnToIndex.getOrDefault(colDesc.name(), colDesc.logicalIndex());
+
+            if (hnd.get(colIdInRow, row) != null) {
+                nonNullCols++;
+            }
+        }
+
+        return nonNullCols;
+    }
+
     private <RowT> BinaryRow deleteTuple(RowT row, ExecutionContext<RowT> ectx) {
         int nonNullVarlenKeyCols = 0;