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

[20/29] ignite git commit: IGNITE-5941 - Fixed index name length restrictions. This closes #2408

IGNITE-5941 - Fixed index name length restrictions. This closes #2408


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

Branch: refs/heads/ignite-5947
Commit: fde550bac56fd0cc7c51c62a9c291dd4c3f3030c
Parents: 74d6ab9
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Mon Aug 14 11:32:11 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Aug 14 11:33:34 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/pagemem/PageUtils.java      |  26 +++
 .../cache/persistence/MetadataStorage.java      |  12 +-
 .../cache/index/LongIndexNameTest.java          | 212 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 4 files changed, 246 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fde550ba/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
index 3fa5954..362ac54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -37,6 +37,19 @@ public class PageUtils {
     }
 
     /**
+     *
+     * @param addr Start address.
+     * @param off Offset.
+     * @return Byte value from given address.
+     */
+    public static int getUnsignedByte(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getByte(addr + off) & 0xFF;
+    }
+
+    /**
      * @param addr Start address.
      * @param off Offset.
      * @param len Bytes length.
@@ -168,6 +181,19 @@ public class PageUtils {
      * @param off Offset.
      * @param v Value.
      */
+    public static void putUnsignedByte(long addr, int off, int v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert v >= 0 && v <= 255;
+
+        GridUnsafe.putByte(addr + off, (byte) v);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
     public static void putShort(long addr, int off, short v) {
         assert addr > 0 : addr;
         assert off >= 0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/fde550ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
index 743f3b9..498ecdd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
@@ -39,7 +39,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
  */
 public class MetadataStorage implements MetaStore {
     /** Max index name length (bytes num) */
-    public static final int MAX_IDX_NAME_LEN = 768;
+    public static final int MAX_IDX_NAME_LEN = 255;
 
     /** Bytes in byte. */
     private static final int BYTE_LEN = 1;
@@ -197,7 +197,7 @@ public class MetadataStorage implements MetaStore {
             int shift = 0;
 
             // Compare index names.
-            final byte len = PageUtils.getByte(pageAddr, off + shift);
+            final int len = PageUtils.getUnsignedByte(pageAddr, off + shift);
 
             shift += BYTE_LEN;
 
@@ -256,7 +256,7 @@ public class MetadataStorage implements MetaStore {
         final IndexItem row
     ) {
         // Index name length.
-        PageUtils.putByte(pageAddr, off, (byte)row.idxName.length);
+        PageUtils.putUnsignedByte(pageAddr, off, row.idxName.length);
         off++;
 
         // Index name.
@@ -282,10 +282,10 @@ public class MetadataStorage implements MetaStore {
         int srcOff
     ) {
         // Index name length.
-        final byte len = PageUtils.getByte(srcPageAddr, srcOff);
+        final int len = PageUtils.getUnsignedByte(srcPageAddr, srcOff);
         srcOff++;
 
-        PageUtils.putByte(dstPageAddr, dstOff, len);
+        PageUtils.putUnsignedByte(dstPageAddr, dstOff, len);
         dstOff++;
 
         PageHandler.copyMemory(srcPageAddr, dstPageAddr, srcOff, dstOff, len);
@@ -305,7 +305,7 @@ public class MetadataStorage implements MetaStore {
      */
     private static IndexItem readRow(final long pageAddr, int off) {
         // Index name length.
-        final int len = PageUtils.getByte(pageAddr, off) & 0xFF;
+        final int len = PageUtils.getUnsignedByte(pageAddr, off) & 0xFF;
         off++;
 
         // Index name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/fde550ba/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java
new file mode 100644
index 0000000..92883a4
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/LongIndexNameTest.java
@@ -0,0 +1,212 @@
+/*
+ * 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.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.NotNull;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+/**
+ *
+ */
+public class LongIndexNameTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setPersistentStoreConfiguration(new PersistentStoreConfiguration())
+            .setCacheConfiguration(new <String, Person>CacheConfiguration("cache")
+                .setQueryEntities(getIndexCfg())
+                .setAffinity(new RendezvousAffinityFunction(false, 16)));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        deleteWorkFiles();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLongIndexNames() throws Exception {
+        try {
+            Ignite ignite = startGrid(0);
+
+            IgniteCache cache = insertSomeData(ignite);
+
+            QueryCursor cursor1 = cache.query(new SqlFieldsQuery("SELECT * FROM Person where name like '%Name 0'"));
+            QueryCursor cursor1Idx = cache.query(new SqlFieldsQuery("SELECT * FROM Person where name = 'Name 0'"));
+
+            QueryCursor cursor2 = cache.query(new SqlFieldsQuery("SELECT * FROM Person where age like '%0'"));
+            QueryCursor cursor2Idx = cache.query(new SqlFieldsQuery("SELECT * FROM Person where age = 0"));
+
+            assertEquals(cursor1.getAll().size(), cursor1Idx.getAll().size());
+            assertEquals(cursor2.getAll().size(), cursor2Idx.getAll().size());
+
+            ignite.close();
+
+            Thread.sleep(2_000);
+
+            ignite = startGrid(0);
+
+            cache = insertSomeData(ignite);
+
+            cursor1 = cache.query(new SqlFieldsQuery("SELECT * FROM Person where name like '%Name 0'"));
+            cursor1Idx = cache.query(new SqlFieldsQuery("SELECT * FROM Person where name = 'Name 0'"));
+
+            cursor2 = cache.query(new SqlFieldsQuery("SELECT * FROM Person where age like '%0'"));
+            cursor2Idx = cache.query(new SqlFieldsQuery("SELECT * FROM Person where age = 0"));
+
+            assertEquals(cursor1.getAll().size(), cursor1Idx.getAll().size());
+            assertEquals(cursor2.getAll().size(), cursor2Idx.getAll().size());
+
+
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     *
+     */
+    @NotNull private IgniteCache insertSomeData(Ignite ignite) {
+        if (!ignite.active())
+            ignite.active(true);
+
+        IgniteCache<String, Person> cache = ignite.cache("cache");
+
+        for (int i=0; i<10; i++)
+            cache.put(String.valueOf(System.currentTimeMillis()), new Person("Name " + i, i));
+
+        return cache;
+    }
+
+    /**
+     *
+     */
+    public static List<QueryEntity> getIndexCfg() {
+        ArrayList<QueryEntity> entities = new ArrayList<>();
+
+        QueryEntity qe = new QueryEntity(String.class.getName(), Person.class.getName());
+
+        LinkedHashMap<String, String> fieldsMap = new LinkedHashMap<>();
+        fieldsMap.put("name", String.class.getName());
+        fieldsMap.put("age", Integer.class.getName());
+
+        qe.setFields(fieldsMap);
+
+        ArrayList<QueryIndex> indices = new ArrayList<>();
+        QueryIndex index = new QueryIndex("name", true, "LONG_NAME_123456789012345678901234567890" +
+            "12345678901234567890123456789012345678901234567890123456789012345678901234567890");
+
+        QueryIndex index2 = new QueryIndex("age", true, "AGE_IDX");
+        indices.add(index);
+        indices.add(index2);
+
+        qe.setIndexes(indices);
+
+        entities.add(qe);
+
+        return entities;
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void deleteWorkFiles() throws IgniteCheckedException {
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+    }
+
+    /**
+     *
+     */
+    private static class Person {
+        /** */
+        private String name;
+
+        /** */
+        private int age;
+
+        /**
+         *
+         */
+        public Person() {
+            // No-op.
+        }
+
+        /**
+         * @param name Name.
+         * @param age Age.
+         */
+        public Person(String name, int age) {
+            this.name = name;
+            this.age = age;
+        }
+
+        /**
+         * @return Name.
+         */
+        public String getName() {
+            return name;
+        }
+
+        /**
+         * @param name Name.
+         */
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        /**
+         * @return Age.
+         */
+        public int getAge() {
+            return age;
+        }
+
+        /**
+         * @param age Age.
+         */
+        public void setAge(int age) {
+            this.age = age;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fde550ba/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 1ad0d4b..decc7d5 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
@@ -108,6 +108,7 @@ import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComple
 import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexServerTransactionalPartitionedTest;
 import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexServerTransactionalReplicatedTest;
 import org.apache.ignite.internal.processors.cache.index.H2DynamicTableSelfTest;
+import org.apache.ignite.internal.processors.cache.index.LongIndexNameTest;
 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;
@@ -317,6 +318,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheDistributedPartitionQueryConfigurationSelfTest.class);
         suite.addTestSuite(IgniteSqlKeyValueFieldsTest.class);
         suite.addTestSuite(IgniteSqlRoutingTest.class);
+        suite.addTestSuite(LongIndexNameTest.class);
 
         return suite;
     }