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/02/16 11:56:45 UTC

[5/7] ignite git commit: Inline offheap indexes (simple types + String).

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f836cb0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java
new file mode 100644
index 0000000..a874ad4
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java
@@ -0,0 +1,132 @@
+/*
+ * 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.h2.database.io;
+
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
+import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.database.InlineIndexHelper;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Row;
+import org.h2.result.SearchRow;
+
+/**
+ * Leaf page for H2 row references.
+ */
+public class H2ExtrasLeafIO extends BPlusLeafIO<SearchRow> {
+
+    /** Payload size. */
+    private final int payloadSize;
+
+    /** */
+    public static void register() {
+        for (short payload = 1; payload <= PageIO.MAX_PAYLOAD_SIZE; payload++)
+            PageIO.registerH2ExtraLeaf(getVersions((short)(PageIO.T_H2_EX_REF_LEAF_START + payload - 1), payload));
+    }
+
+    /**
+     * @param payload Payload size.
+     * @return IOVersions for given payload.
+     */
+    public static IOVersions<? extends BPlusLeafIO<SearchRow>> getVersions(int payload) {
+        assert payload >= 0 && payload <= PageIO.MAX_PAYLOAD_SIZE;
+
+        if (payload == 0)
+            return H2LeafIO.VERSIONS;
+        else
+            return (IOVersions<BPlusLeafIO<SearchRow>>)PageIO.getLeafVersions((short)(payload - 1));
+    }
+
+    /** */
+    private static IOVersions<H2ExtrasLeafIO> getVersions(short type, short payload) {
+        return new IOVersions<>(new H2ExtrasLeafIO(type, 1, payload));
+    }
+
+    /**
+     * @param type Page type.
+     * @param ver Page format version.
+     * @param payloadSize Payload size.
+     */
+    private H2ExtrasLeafIO(short type, int ver, int payloadSize) {
+        super(type, ver, 8 + payloadSize);
+        this.payloadSize = payloadSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void storeByOffset(long pageAddr, int off, SearchRow row) {
+        GridH2Row row0 = (GridH2Row)row;
+
+        assert row0.link != 0;
+
+        H2TreeIndex currentIdx = H2TreeIndex.getCurrentIndex();
+        assert currentIdx != null;
+        List<InlineIndexHelper> inlineIdxs = currentIdx.inlineIndexes();
+
+        assert inlineIdxs != null;
+
+        int fieldOff = 0;
+
+        for (int i = 0; i < inlineIdxs.size(); i++) {
+            InlineIndexHelper idx = inlineIdxs.get(i);
+            int size = idx.put(pageAddr, off + fieldOff, row.getValue(idx.columnIndex()), payloadSize - fieldOff);
+            if (size == 0)
+                break;
+            fieldOff += size;
+        }
+
+        PageUtils.putLong(pageAddr, off + payloadSize, row0.link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<SearchRow> srcIo, long srcPageAddr, int srcIdx) {
+        int srcOff = srcIo.offset(srcIdx);
+
+        byte[] payload = PageUtils.getBytes(srcPageAddr, srcOff, payloadSize);
+        long link = PageUtils.getLong(srcPageAddr, srcOff + payloadSize);
+
+        assert link != 0;
+
+        int dstOff = offset(dstIdx);
+
+        PageUtils.putBytes(dstPageAddr, dstOff, payload);
+        PageUtils.putLong(dstPageAddr, dstOff + payloadSize, link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public SearchRow getLookupRow(BPlusTree<SearchRow, ?> tree, long pageAddr, int idx)
+        throws IgniteCheckedException {
+        long link = getLink(pageAddr, idx);
+
+        return ((H2Tree)tree).getRowFactory().getRow(link);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param idx Index.
+     * @return Link to row.
+     */
+    private long getLink(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx) + payloadSize);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f836cb0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
index bdd43ef..6d4e97a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
@@ -19,8 +19,8 @@ package org.apache.ignite.internal.processors.query.h2.opt;
 
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.h2.result.Row;
 import org.h2.result.SearchRow;
 import org.h2.store.Data;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f836cb0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
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 5024959..f3069a7 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
@@ -34,7 +34,8 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.query.h2.database.*;
+import org.apache.ignite.internal.processors.query.h2.database.H2RowFactory;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteBiTuple;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f836cb0/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index b9152fa..133ae4e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -500,6 +500,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         @Override public GridQueryIndexType type() {
             return GridQueryIndexType.FULLTEXT;
         }
+
+        /** {@inheritDoc} */
+        @Override public int inlineSize() {
+            return 0;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f836cb0/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndexTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndexTest.java
new file mode 100644
index 0000000..064da45
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndexTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.h2.database;
+
+import junit.framework.TestCase;
+import org.h2.result.SortOrder;
+import org.h2.value.Value;
+import org.h2.value.ValueNull;
+import org.h2.value.ValueString;
+
+/**
+ * Simple tests for {@link H2TreeIndex}.
+ */
+public class H2TreeIndexTest extends TestCase {
+
+    /** Test on String values compare */
+    public void testRelyOnCompare() {
+
+        InlineIndexHelper ha = new InlineIndexHelper(Value.STRING, 0, SortOrder.ASCENDING);
+        InlineIndexHelper hd = new InlineIndexHelper(Value.STRING, 0, SortOrder.DESCENDING);
+
+        // same size
+        assertFalse(H2TreeIndex.canRelyOnCompare(0, ValueString.get("aabb"), ValueString.get("aabb"), ha));
+        assertFalse(H2TreeIndex.canRelyOnCompare(0, ValueString.get("aabb"), ValueString.get("aabb"), hd));
+
+        // second string is shorter
+        assertTrue(H2TreeIndex.canRelyOnCompare(1, ValueString.get("aabb"), ValueString.get("aab"), ha));
+        assertTrue(H2TreeIndex.canRelyOnCompare(-1, ValueString.get("aabb"), ValueString.get("aab"), hd));
+
+        // second string is longer
+        assertTrue(H2TreeIndex.canRelyOnCompare(1, ValueString.get("aabb"), ValueString.get("aaaaaa"), ha));
+        assertTrue(H2TreeIndex.canRelyOnCompare(-1, ValueString.get("aabb"), ValueString.get("aaaaaa"), hd));
+
+        assertFalse(H2TreeIndex.canRelyOnCompare(-1, ValueString.get("aab"), ValueString.get("aabbbbb"), ha));
+        assertFalse(H2TreeIndex.canRelyOnCompare(1, ValueString.get("aab"), ValueString.get("aabbbbb"), hd));
+
+        // one is null
+        assertTrue(H2TreeIndex.canRelyOnCompare(1, ValueString.get("aabb"), ValueNull.INSTANCE, ha));
+        assertTrue(H2TreeIndex.canRelyOnCompare(-1, ValueNull.INSTANCE, ValueString.get("aab"), ha));
+        assertTrue(H2TreeIndex.canRelyOnCompare(0, ValueNull.INSTANCE, ValueNull.INSTANCE, ha));
+
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f836cb0/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
new file mode 100644
index 0000000..f65f0fa
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.h2.database;
+
+import junit.framework.TestCase;
+
+/**
+ * Simple tests for {@link InlineIndexHelper}.
+ */
+public class InlineIndexHelperTest extends TestCase {
+
+    /** Test utf-8 string cutting. */
+    public void testConvert() {
+        // 8 bytes total: 1b, 1b, 3b, 3b.
+
+        byte[] bytes = InlineIndexHelper.toBytes("00\u20ac\u20ac", 7);
+        assertEquals(5, bytes.length);
+
+        String s = new String(bytes);
+        assertEquals(3, s.length());
+    }
+
+    /** Limit is too small to cut */
+    public void testShort() {
+        // 6 bytes total: 3b, 3b.
+
+        byte[] bytes = InlineIndexHelper.toBytes("\u20ac\u20ac", 2);
+        assertNull(bytes);
+    }
+
+}
\ No newline at end of file