You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/12/27 08:24:18 UTC

[11/23] ignite git commit: IGNITE-6915: Fixed B+Tree index names for non-binary marshaller case. This closes #3183.

IGNITE-6915: Fixed B+Tree index names for non-binary marshaller case. This closes #3183.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7722b97c
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7722b97c
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7722b97c

Branch: refs/heads/ignite-zk
Commit: 7722b97cb3d529318e7536070a4c783c54f662ca
Parents: d36c171
Author: shq <ki...@gmail.com>
Authored: Tue Dec 26 19:12:03 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Dec 26 19:12:03 2017 +0300

----------------------------------------------------------------------
 .../query/h2/database/H2TreeIndex.java          |   7 +-
 .../index/OptimizedMarshallerIndexNameTest.java | 269 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +-
 3 files changed, 277 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7722b97c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
----------------------------------------------------------------------
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 53d5de5..18d9bec 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
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.persistence.RootPage;
 import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.h2.H2Cursor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
 import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
@@ -101,7 +102,11 @@ public class H2TreeIndex extends GridH2IndexBase {
         initBaseIndex(tbl, 0, name, cols,
             pk ? IndexType.createPrimaryKey(false, false) : IndexType.createNonUnique(false, false, false));
 
-        name = (tbl.rowDescriptor() == null ? "" : tbl.rowDescriptor().type().typeId() + "_") + name;
+        GridQueryTypeDescriptor typeDesc = tbl.rowDescriptor().type();
+
+        int typeId = cctx.binaryMarshaller() ? typeDesc.typeId() : typeDesc.valueClass().hashCode();
+
+        name = (tbl.rowDescriptor() == null ? "" : typeId + "_") + name;
 
         name = BPlusTree.treeName(name, "H2Tree");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7722b97c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java
new file mode 100644
index 0000000..db732eb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/OptimizedMarshallerIndexNameTest.java
@@ -0,0 +1,269 @@
+/*
+ * 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.cache.index;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+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.configuration.NearCacheConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.IgniteTestResources;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Verifies correct indexes naming for Optimized Marshaller with enabled persistence case.
+ *
+ * See IGNITE-6915 for details.
+ */
+
+public class OptimizedMarshallerIndexNameTest extends GridCommonAbstractTest {
+
+    /** Test name 1 */
+    private static final String TEST_NAME1 = "Name1";
+    /** Test name 2 */
+    private static final String TEST_NAME2 = "Name2";
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName, IgniteTestResources rsrcs)
+        throws Exception {
+
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName, rsrcs);
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration().setMaxSize(300 * 1024 * 1024).setPersistenceEnabled(true))
+            .setStoragePath(workSubdir() + "/db")
+            .setWalArchivePath(workSubdir() + "/db/wal/archive")
+            .setWalPath(workSubdir() + "/db/wal")
+            .setWalMode(WALMode.LOG_ONLY);
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        cfg.setMarshaller(new OptimizedMarshaller());
+
+        return cfg;
+    }
+
+    /**
+     * Creates cache configuration with required indexed types.
+     *
+     * @param name The name of the cache
+     */
+    @SuppressWarnings("deprecation")
+    protected static CacheConfiguration<Object, Object> cacheConfiguration(String name) {
+        CacheConfiguration<Object, Object> cfg = new CacheConfiguration<>(name);
+
+        cfg.setAtomicityMode(TRANSACTIONAL);
+        cfg.setNearConfiguration(new NearCacheConfiguration<>());
+        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setEvictionPolicy(null);
+
+        cfg.setIndexedTypes(
+            UUID.class, Person.class,
+            UUID.class, FalsePerson.class);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), workSubdir(), true));
+
+        startGrid(getTestIgniteInstanceName());
+        grid().active(true);
+    }
+
+    /**
+     * Verifies that BPlusTree are not erroneously shared between tables in the same cache
+     * due to IGNITE-6915 bug.
+     */
+    public void testOptimizedMarshallerIndex() {
+
+        // Put objects of different types into the same cache
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration("PersonEn");
+
+        IgniteCache<Object, Object> cache = grid().getOrCreateCache(ccfg);
+
+        cache.put(UUID.randomUUID(), new Person(TEST_NAME1, 42));
+        cache.put(UUID.randomUUID(), new FalsePerson(32, TEST_NAME2));
+
+        // Run query against one particular type
+        SqlFieldsQueryEx qry = new SqlFieldsQueryEx(
+            "select * from " + QueryUtils.typeName(FalsePerson.class), true);
+
+        // If fix for IGNITE-6915 doesn't work you should see exception like the one below in the log:
+        //
+        // org.h2.jdbc.JdbcSQLException: General error: "class org.apache.ignite.IgniteCheckedException:
+        // Failed to invoke getter method [type=int, property=name,
+        // obj=org.apache.ignite.internal.processors.cache.index.OptimizedMarshallerIndexNameTest$Person@...:
+        // org.apache.ignite.internal.processors.cache.index.OptimizedMarshallerIndexNameTest$Person@...,
+        // getter=public int org.apache.ignite.internal.processors.cache.index.OptimizedMarshallerIndexNameTest$FalsePerson.getName()]"
+
+        List<List<?>> res = cache.query(qry).getAll();
+
+        assertEquals(1, res.size());
+    }
+
+    /**
+     * Returns subdirectory of the work directory to put persistence store.
+     * For this test it's a class name.
+     *
+     * @return The name of subdirectory (the short name of the test class).
+     */
+    @NotNull private String workSubdir() {
+        return getClass().getSimpleName();
+    }
+
+    /** Entity to query. */
+    public static class Person implements Externalizable {
+
+        /** Person name. */
+        @QuerySqlField(index = true, inlineSize = 0)
+        private String name;
+
+        /** Person age. */
+        @QuerySqlField(index = true, inlineSize = 0)
+        private int age;
+
+        /** Creates a unnamed newborn person. */
+        public Person() {
+        }
+
+        /**
+         * Creates a person.
+         *
+         * @param name Name
+         * @param age Age
+         */
+        public Person(String name, int age) {
+            this.name = name;
+            this.age = age;
+        }
+
+        /**
+         * Returns name of the person.
+         * @return The name of the person.
+         */
+        public String getName() {
+            return name;
+        }
+
+        /**
+         * Returns age of the person.
+         * @return Person's age.
+         */
+        public int getAge() {
+            return age;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeUTF(name);
+            out.writeInt(age);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException {
+            name = in.readUTF();
+            age = in.readInt();
+        }
+    }
+
+    /**
+     * The class that should not be met in the Person table queried
+     * due to mixing of {@link BPlusTree}-s.
+     *
+     * Note that the types of name and age are swapped.
+     */
+    public static class FalsePerson implements Externalizable {
+
+        /** Person numeric name in future digital age */
+        @QuerySqlField(index = true, inlineSize = 0)
+        private int name;
+
+        /** Age is a string. Life's road could be twisted. */
+        @QuerySqlField(index = true, inlineSize = 0)
+        private String age;
+
+        /** Creates an anonymous baby. */
+        public FalsePerson() {
+        }
+
+        /**
+         * Creates a person of new type.
+         *
+         * @param name Numeric name.
+         * @param age Digital age.
+         */
+        public FalsePerson(int name, String age) {
+            this.name = name;
+            this.age = age;
+        }
+
+        /**
+         * Says how should you call this person.
+         *
+         * @return that digital name of the person.
+         */
+        public int getName() {
+            return name;
+        }
+
+        /**
+         * Makes you informed about person's bio.
+         *
+         * @return age as a string.
+         */
+        public String getAge() {
+            return age;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(name);
+            out.writeUTF(age);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException {
+            name = in.readInt();
+            age = in.readUTF();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7722b97c/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 58b8f0a..55d0b5c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -123,6 +123,7 @@ import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComple
 import org.apache.ignite.internal.processors.cache.index.H2DynamicTableSelfTest;
 import org.apache.ignite.internal.processors.cache.index.H2ConnectionLeaksSelfTest;
 import org.apache.ignite.internal.processors.cache.index.LongIndexNameTest;
+import org.apache.ignite.internal.processors.cache.index.OptimizedMarshallerIndexNameTest;
 import org.apache.ignite.internal.processors.cache.index.SchemaExchangeSelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
@@ -372,8 +373,8 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheQuerySqlFieldInlineSizeSelfTest.class);
         suite.addTestSuite(IgniteSqlParameterizedQueryTest.class);
         suite.addTestSuite(H2ConnectionLeaksSelfTest.class);
-
         suite.addTestSuite(IgniteCheckClusterStateBeforeExecuteQueryTest.class);
+        suite.addTestSuite(OptimizedMarshallerIndexNameTest.class);
 
         return suite;
     }