You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tl...@apache.org on 2020/07/09 13:54:20 UTC

[ignite] branch master updated: IGNITE-13200 SQL: fix node stops on create index on invalid data type (#7982)

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

tledkov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 3dd359b  IGNITE-13200 SQL: fix node stops on create index on invalid data type (#7982)
3dd359b is described below

commit 3dd359b38d1176c9101d3887c90c51cc8a95d85f
Author: tledkov <tl...@gridgain.com>
AuthorDate: Thu Jul 9 16:52:55 2020 +0300

    IGNITE-13200 SQL: fix node stops on create index on invalid data type (#7982)
---
 .../cache/CachePartialUpdateCheckedException.java  |   7 +
 .../cache/query/IgniteQueryErrorCode.java          |   3 +
 .../processors/query/h2/database/H2TreeIndex.java  |  12 ++
 .../processors/query/h2/opt/GridH2Table.java       |  46 ++++-
 .../processors/query/h2/opt/H2CacheRow.java        |  19 +-
 .../query/CreateIndexOnInvalidDataTypeTest.java    | 207 +++++++++++++++++++++
 .../IgniteBinaryCacheQueryTestSuite2.java          |   3 +
 7 files changed, 286 insertions(+), 11 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
index ab62ed2..3fc9cdb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachePartialUpdateCheckedException.java
@@ -45,6 +45,13 @@ public class CachePartialUpdateCheckedException extends IgniteCheckedException {
     }
 
     /**
+     * @param msg Error message.
+     */
+    public CachePartialUpdateCheckedException(String msg, Throwable cause) {
+        super(msg, cause);
+    }
+
+    /**
      * Gets collection of failed keys.
      * @return Collection of failed keys.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
index bf8c464..705be74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
@@ -135,6 +135,9 @@ public final class IgniteQueryErrorCode {
     /** Transaction serialization error. */
     public static final int TRANSACTION_SERIALIZATION_ERROR = 5005;
 
+    /** Field type mismatch. e.g.: cause is {@link ClassCastException}. */
+    public static final int FIELD_TYPE_MISMATCH = 5006;
+
     /** */
     private IgniteQueryErrorCode() {
         // No-op.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
index 735ae03..b2d770e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
@@ -48,6 +48,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.failure.FailureProcessor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.h2.DurableBackgroundCleanupIndexTreeTask;
 import org.apache.ignite.internal.processors.query.h2.H2Cursor;
 import org.apache.ignite.internal.processors.query.h2.H2RowCache;
@@ -420,8 +421,19 @@ public class H2TreeIndex extends H2TreeIndexBase {
         }
     }
 
+    /**
+     * @param row Row to validate.
+     * @throws IgniteSQLException on error (field type mismatch).
+     */
+    private void validateRowFields(H2CacheRow row) {
+        for (int col : columnIds)
+            row.getValue(col);
+    }
+
     /** {@inheritDoc} */
     @Override public boolean putx(H2CacheRow row) {
+        validateRowFields(row);
+
         try {
             int seg = segmentForRow(cctx, row);
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index b1ca683..a6a3e16 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -42,6 +42,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.cache.query.QueryTable;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryField;
@@ -56,6 +57,7 @@ import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndexBase;
 import org.apache.ignite.internal.processors.query.h2.database.IndexInformation;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
@@ -751,6 +753,8 @@ public class GridH2Table extends TableBase {
         if (prevRow0 != null)
             prevRow0.prepareValuesCache();
 
+        IgniteCheckedException err = null;
+
         try {
             lock(false);
 
@@ -774,12 +778,12 @@ public class GridH2Table extends TableBase {
                     Index idx = idxs.get(i);
 
                     if (idx instanceof GridH2IndexBase)
-                        addToIndex((GridH2IndexBase)idx, row0, prevRow0);
+                        err = addToIndex((GridH2IndexBase)idx, row0, prevRow0, err);
                 }
 
                 if (!tmpIdxs.isEmpty()) {
                     for (GridH2IndexBase idx : tmpIdxs.values())
-                        addToIndex(idx, row0, prevRow0);
+                        err = addToIndex(idx, row0, prevRow0, err);
                 }
             }
             finally {
@@ -792,6 +796,9 @@ public class GridH2Table extends TableBase {
             if (prevRow0 != null)
                 prevRow0.clearValuesCache();
         }
+
+        if (err != null)
+            throw err;
     }
 
     /**
@@ -839,13 +846,38 @@ public class GridH2Table extends TableBase {
      * @param idx Index to add row to.
      * @param row Row to add to index.
      * @param prevRow Previous row state, if any.
+     * @param prevErr Error on index add.
      */
-    private void addToIndex(GridH2IndexBase idx, H2CacheRow row, H2CacheRow prevRow) {
-        boolean replaced = idx.putx(row);
+    private IgniteCheckedException addToIndex(
+        GridH2IndexBase idx,
+        H2CacheRow row,
+        H2CacheRow prevRow,
+        IgniteCheckedException prevErr
+    ) {
+        try {
+            boolean replaced = idx.putx(row);
+
+            // Row was not replaced, need to remove manually.
+            if (!replaced && prevRow != null)
+                idx.removex(prevRow);
+
+            return prevErr;
+        }
+        catch (Throwable t) {
+            IgniteSQLException ex = X.cause(t, IgniteSQLException.class);
+
+            if (ex != null && ex.statusCode() == IgniteQueryErrorCode.FIELD_TYPE_MISMATCH) {
+                if (prevErr != null) {
+                    prevErr.addSuppressed(t);
 
-        // Row was not replaced, need to remove manually.
-        if (!replaced && prevRow != null)
-            idx.removex(prevRow);
+                    return prevErr;
+                }
+                else
+                    return new IgniteCheckedException("Error on add row to index '" + getName() + '\'', t);
+            }
+            else
+                throw t;
+        }
     }
 
     /**
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java
index c302aa8..527a33e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java
@@ -22,8 +22,10 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.H2Utils;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -171,6 +173,10 @@ public class H2CacheRow extends H2Row implements CacheDataRow {
         try {
             return H2Utils.wrap(desc.indexing().objectContext(), val, type);
         }
+        catch (ClassCastException e) {
+            throw new IgniteSQLException("Failed to wrap object into H2 Value. " + e.getMessage(),
+                IgniteQueryErrorCode.FIELD_TYPE_MISMATCH, e);
+        }
         catch (IgniteCheckedException e) {
             throw new IgniteException("Failed to wrap object into H2 Value.", e);
         }
@@ -320,13 +326,18 @@ public class H2CacheRow extends H2Row implements CacheDataRow {
 
         if (v != null) {
             for (int i = QueryUtils.DEFAULT_COLUMNS_COUNT, cnt = getColumnCount(); i < cnt; i++) {
-                v = getValue(i);
-
                 if (i != QueryUtils.DEFAULT_COLUMNS_COUNT)
                     sb.a(", ");
 
-                if (!desc.isKeyValueOrVersionColumn(i))
-                    sb.a(v == null ? "nil" : (S.includeSensitive() ? v.getString() : "data hidden"));
+                try {
+                    v = getValue(i);
+
+                    if (!desc.isKeyValueOrVersionColumn(i))
+                        sb.a(v == null ? "nil" : (S.includeSensitive() ? v.getString() : "data hidden"));
+                }
+                catch (Exception e) {
+                    sb.a("<value skipped on error: " + e.getMessage() + '>');
+                }
             }
         }
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/CreateIndexOnInvalidDataTypeTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/CreateIndexOnInvalidDataTypeTest.java
new file mode 100644
index 0000000..bead5e4
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/CreateIndexOnInvalidDataTypeTest.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query;
+
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
+import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
+import org.apache.ignite.internal.processors.cache.index.AbstractIndexingCommonTest;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+/**
+ * Checks add field with invalid data type to index.
+ */
+public class CreateIndexOnInvalidDataTypeTest extends AbstractIndexingCommonTest {
+    /** Keys count. */
+    private static final int KEY_CNT = 10;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setFailureHandler(new StopNodeFailureHandler())
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDefaultDataRegionConfiguration(
+                        new DataRegionConfiguration()
+                            .setPersistenceEnabled(true)
+                    )
+            );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * Check case when index is created on the field with invalid data type.
+     * Test steps:
+     * - create cache with query entity describes a table;
+     * - fill data (real data contains the fields that was not described by query entity);
+     * - execute alter table (ADD COLUMN with invalid type for exists field);
+     * - try to create index for the new field - exception must be throw;
+     * - checks that index isn't created.
+     */
+    @Test
+    public void testCreateIndexOnInvalidData() throws Exception {
+        startGrid();
+
+        grid().cluster().state(ClusterState.ACTIVE);
+
+        IgniteCache<Integer, Value> c = grid().createCache(
+            new CacheConfiguration<Integer, Value>()
+                .setName("test")
+                .setSqlSchema("PUBLIC")
+                .setQueryEntities(
+                    Collections.singleton(
+                        new QueryEntity(Integer.class, Value.class)
+                            .setTableName("TEST")
+                    )
+                )
+                .setBackups(1)
+                .setAffinity(new RendezvousAffinityFunction(false, 10))
+        );
+
+        for (int i = 0; i < KEY_CNT; ++i)
+            c.put(i, new Value(i));
+
+        sql("ALTER TABLE TEST ADD COLUMN (VAL_DATE DATE)");
+
+        sql("CREATE INDEX TEST_VAL_INT_IDX ON TEST(VAL_INT)");
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> {
+                sql("CREATE INDEX TEST_VAL_DATE_IDX ON TEST(VAL_DATE)");
+
+                return null;
+            },
+            IgniteSQLException.class, "java.util.Date cannot be cast to java.sql.Date");
+
+        // Wait for node stop if it is initiated by FailureHandler
+        U.sleep(1000);
+
+        List<List<?>> res = sql("SELECT val_int FROM TEST where val_int > -1").getAll();
+
+        assertEquals(KEY_CNT, res.size());
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> {
+                sql("DROP INDEX TEST_VAL_DATE_IDX");
+
+                return null;
+            },
+            IgniteSQLException.class, "Index doesn't exist: TEST_VAL_DATE_IDX");
+    }
+
+    /**
+     * Check case when row with invalid field is added.
+     * Test steps:
+     * - create table;
+     * - create two index;
+     * - try add entry - exception must be thrown;
+     * - remove the index for field with invalid type;
+     * - check that select query that uses the index for valid field is successful.
+     */
+    @Test
+    public void testAddInvalidDataToIndex() throws Exception {
+        startGrid();
+
+        grid().cluster().state(ClusterState.ACTIVE);
+
+        sql("CREATE TABLE TEST (ID INT PRIMARY KEY, VAL_INT INT, VAL_DATE DATE) " +
+            "WITH \"CACHE_NAME=test,VALUE_TYPE=ValueType0\"");
+
+        sql("CREATE INDEX TEST_0_VAL_DATE_IDX ON TEST(VAL_DATE)");
+        sql("CREATE INDEX TEST_1_VAL_INT_IDX ON TEST(VAL_INT)");
+
+        BinaryObjectBuilder bob = grid().binary().builder("ValueType0");
+
+        bob.setField("VAL_INT", 10);
+        bob.setField("VAL_DATE", new java.util.Date());
+
+        GridTestUtils.assertThrowsAnyCause(log, () -> {
+                grid().cache("test").put(0, bob.build());
+
+                return null;
+            },
+            CachePartialUpdateCheckedException.class, "Failed to update keys");
+
+        sql("DROP INDEX TEST_0_VAL_DATE_IDX");
+
+        // Check successful insert after index is dropped.
+        grid().cache("test").put(1, bob.build());
+
+        List<List<?>> res = sql("SELECT VAL_INT FROM TEST WHERE VAL_INT > 0").getAll();
+
+        assertEquals(2, res.size());
+    }
+
+    /**
+     * @param sql SQL query.
+     * @param args Query parameters.
+     * @return Results cursor.
+     */
+    private FieldsQueryCursor<List<?>> sql(String sql, Object... args) {
+        return grid().context().query().querySqlFields(new SqlFieldsQuery(sql)
+            .setLazy(true)
+            .setArgs(args), false);
+    }
+
+    /**
+     *
+     */
+    private static class Value {
+        /** */
+        @QuerySqlField
+        int val_int;
+
+        /** */
+        java.util.Date val_date;
+
+        /**
+         * @param val Test value.
+         */
+        public Value(int val) {
+            this.val_int = val;
+            val_date = new Date(val);
+        }
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
index d09c081..1c99436 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryDe
 import org.apache.ignite.internal.processors.cache.query.ScanQueryOffheapExpiryPolicySelfTest;
 import org.apache.ignite.internal.processors.database.baseline.IgniteChangingBaselineCacheQueryNodeRestartSelfTest;
 import org.apache.ignite.internal.processors.database.baseline.IgniteStableBaselineCacheQueryNodeRestartsSelfTest;
+import org.apache.ignite.internal.processors.query.CreateIndexOnInvalidDataTypeTest;
 import org.apache.ignite.internal.processors.query.DisabledSqlFunctionsTest;
 import org.apache.ignite.internal.processors.query.DmlBatchSizeDeadlockTest;
 import org.apache.ignite.internal.processors.query.IgniteCacheGroupsCompareQueryTest;
@@ -88,6 +89,8 @@ import org.junit.runners.Suite;
  */
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
+    CreateIndexOnInvalidDataTypeTest.class,
+
     DisabledSqlFunctionsTest.class,
 
     SqlCacheStartStopTest.class,