Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id CE03E19665 for ; Thu, 28 Apr 2016 08:52:34 +0000 (UTC) Received: (qmail 41204 invoked by uid 500); 28 Apr 2016 08:52:34 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 40876 invoked by uid 500); 28 Apr 2016 08:52:34 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 39661 invoked by uid 99); 28 Apr 2016 08:52:32 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 28 Apr 2016 08:52:32 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2CCF7E983A; Thu, 28 Apr 2016 08:52:32 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Thu, 28 Apr 2016 08:53:08 -0000 Message-Id: <0a69fa5678ef4ff48ad1d28fde78fe5a@git.apache.org> In-Reply-To: <66d7caab4e4341d5854ef938544c84dd@git.apache.org> References: <66d7caab4e4341d5854ef938544c84dd@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [38/50] ignite git commit: ignite-db - reuse bag ignite-db - reuse bag Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/83d7a5ca Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/83d7a5ca Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/83d7a5ca Branch: refs/heads/ignite-db-x-10884 Commit: 83d7a5caee33e0e77a8ef5ef0eb9ada80b7da760 Parents: bbcf8fe Author: S.Vladykin Authored: Wed Apr 27 05:46:56 2016 +0300 Committer: S.Vladykin Committed: Wed Apr 27 05:46:56 2016 +0300 ---------------------------------------------------------------------- .../cache/database/freelist/FreeList.java | 2 +- .../cache/database/tree/BPlusTree.java | 163 ++++++++++++------- .../cache/database/tree/reuse/ReuseBag.java | 35 ++++ .../cache/database/tree/reuse/ReuseList.java | 33 +++- 4 files changed, 170 insertions(+), 63 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/83d7a5ca/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeList.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeList.java index 68780c2..30ed519 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeList.java @@ -122,7 +122,7 @@ public class FreeList { * @throws IgniteCheckedException If failed. */ private FreeItem take(FreeTree tree, FreeItem lookupItem) throws IgniteCheckedException { - FreeItem res = tree.removeCeil(lookupItem); + FreeItem res = tree.removeCeil(lookupItem, null); assert res == null || (res.pageId() != 0 && res.cacheId() == cctx.cacheId()): res; http://git-wip-us.apache.org/repos/asf/ignite/blob/83d7a5ca/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java index df853c0..c0144a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache.database.tree; import java.nio.ByteBuffer; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -36,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO; import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusMetaIO; import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO; +import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseBag; import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList; import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler; import org.apache.ignite.internal.util.lang.GridCursor; @@ -278,7 +280,7 @@ public abstract class BPlusTree { return Put.RETRY; // Do insert. - L moveUpRow = insert(p.meta, io, buf, p.row, idx, p.rightId, lvl); + L moveUpRow = insert(p.bag, p.meta, io, buf, p.row, idx, p.rightId, lvl); // Check if split happened. if (moveUpRow != null) { @@ -526,7 +528,7 @@ public abstract class BPlusTree { io.initNewPage(buf, metaPageId); - try (Page root = allocatePage()) { + try (Page root = allocatePage(null)) { latestLeafIO().initNewPage(root.getForInitialWrite(), root.id()); io.setLevelsCount(buf, 1); @@ -811,13 +813,14 @@ public abstract class BPlusTree { /** * @param row Lookup row. + * @param bag Reuse bag. * @return Removed row. * @throws IgniteCheckedException If failed. */ - public final T removeCeil(L row) throws IgniteCheckedException { + public final T removeCeil(L row, ReuseBag bag) throws IgniteCheckedException { assert row != null; - return doRemove(row, true); + return doRemove(row, true, bag); } /** @@ -828,17 +831,18 @@ public abstract class BPlusTree { public final T remove(L row) throws IgniteCheckedException { assert row != null; - return doRemove(row, false); + return doRemove(row, false, null); } /** * @param row Lookup row. * @param ceil If we can remove ceil row when we can not find exact. + * @param bag Reuse bag. * @return Removed row. * @throws IgniteCheckedException If failed. */ - private T doRemove(L row, boolean ceil) throws IgniteCheckedException { - Remove r = new Remove(row, ceil); + private T doRemove(L row, boolean ceil, ReuseBag bag) throws IgniteCheckedException { + Remove r = new Remove(row, ceil, bag); try { for (;;) { @@ -865,7 +869,7 @@ public abstract class BPlusTree { r.releaseTail(); r.releaseMeta(); - r.reuseEmptyPages(); + r.reuseFreePages(); } } @@ -1057,7 +1061,17 @@ public abstract class BPlusTree { * @throws IgniteCheckedException If failed. */ public final T put(T row) throws IgniteCheckedException { - Put p = new Put(row); + return put(row, null); + } + + /** + * @param row Row. + * @param bag Reuse bag. + * @return Old row. + * @throws IgniteCheckedException If failed. + */ + public final T put(T row, ReuseBag bag) throws IgniteCheckedException { + Put p = new Put(row, bag); try { for (;;) { // Go down with retries. @@ -1141,6 +1155,7 @@ public abstract class BPlusTree { } /** + * @param bag Reuse bag. * @param meta Meta page. * @param io IO. * @param buf Buffer. @@ -1151,9 +1166,9 @@ public abstract class BPlusTree { * @return Move up row. * @throws IgniteCheckedException If failed. */ - private L insertWithSplit(Page meta, BPlusIO io, final ByteBuffer buf, L row, + private L insertWithSplit(ReuseBag bag, Page meta, BPlusIO io, final ByteBuffer buf, L row, int idx, long rightId, int lvl) throws IgniteCheckedException { - try (Page fwd = allocatePage()) { + try (Page fwd = allocatePage(bag)) { // Need to check this before the actual split, because after the split we will have new forward page here. boolean hadFwd = io.getForward(buf) != 0; @@ -1186,7 +1201,7 @@ public abstract class BPlusTree { if (!hadFwd && lvl == getRootLevel(meta)) { // We are splitting root. long newRootId; - try (Page newRoot = allocatePage()) { + try (Page newRoot = allocatePage(bag)) { newRootId = newRoot.id(); if (io.isLeaf()) @@ -1215,6 +1230,7 @@ public abstract class BPlusTree { } /** + * @param bag Reuse bag. * @param meta Meta page. * @param io IO. * @param buf Buffer. @@ -1225,13 +1241,13 @@ public abstract class BPlusTree { * @return Move up row. * @throws IgniteCheckedException If failed. */ - private L insert(Page meta, BPlusIO io, ByteBuffer buf, L row, int idx, long rightId, int lvl) + private L insert(ReuseBag bag, Page meta, BPlusIO io, ByteBuffer buf, L row, int idx, long rightId, int lvl) throws IgniteCheckedException { int maxCnt = io.getMaxCount(buf); int cnt = io.getCount(buf); if (cnt == maxCnt) // Need to split page. - return insertWithSplit(meta, io, buf, row, idx, rightId, lvl); + return insertWithSplit(bag, meta, io, buf, row, idx, rightId, lvl); insertSimple(io, buf, row, idx, rightId); @@ -1595,11 +1611,17 @@ public abstract class BPlusTree { /** */ byte needReplaceInner = FALSE; + /** */ + ReuseBag bag; + /** * @param row Row. + * @param bag Reuse bag. */ - private Put(T row) { + private Put(T row, ReuseBag bag) { super(row); + + this.bag = bag; } /** {@inheritDoc} */ @@ -1657,7 +1679,7 @@ public abstract class BPlusTree { /** * Remove operation. */ - private final class Remove extends Get { + private final class Remove extends Get implements ReuseBag { /** */ boolean ceil; @@ -1665,9 +1687,6 @@ public abstract class BPlusTree { Tail tail; /** */ - Object emptyPages; - - /** */ byte needReplaceInner = FALSE; /** */ @@ -1682,14 +1701,72 @@ public abstract class BPlusTree { /** */ short innerIdx = Short.MIN_VALUE; + /** */ + Object freePages; + + /** */ + ReuseBag bag; + /** * @param row Row. * @param ceil If we can remove ceil row when we can not find exact. */ - private Remove(L row, boolean ceil) { + private Remove(L row, boolean ceil, ReuseBag bag) { super(row); this.ceil = ceil; + this.bag = bag; + } + + /** + * @return Reuse bag. + */ + private ReuseBag bag() { + return bag != null ? bag : this; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public FullPageId pollFreePage() { + assert bag == null; + + if (freePages == null) + return null; + + if (freePages.getClass() == ArrayDeque.class) + return ((ArrayDeque)freePages).poll(); + + FullPageId res = (FullPageId)freePages; + + freePages = null; + + return res; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public void addFreePage(FullPageId pageId) { + assert pageId != null; + assert bag == null; + + if (freePages == null) + freePages = pageId; + else { + ArrayDeque queue; + + if (freePages.getClass() == ArrayDeque.class) + queue = (ArrayDeque)freePages; + else { + assert freePages instanceof FullPageId; + + queue = new ArrayDeque<>(); + + queue.add(freePages); + freePages = queue; + } + + queue.add(pageId); + } } /** {@inheritDoc} */ @@ -2029,44 +2106,17 @@ public abstract class BPlusTree { if (release) writeUnlockAndClose(page); - if (reuseList == null) - return; // We are not allowed to reuse pages. - - // Reuse empty page. - if (emptyPages == null) - emptyPages = page.fullId(); - else { - List list; - - if (emptyPages.getClass() != ArrayList.class) { - list = new ArrayList<>(5); - - list.add((FullPageId)emptyPages); - - emptyPages = list; - } - else - list = (List)emptyPages; - - list.add(page.fullId()); - } + bag().addFreePage(page.fullId()); } /** * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") - private void reuseEmptyPages() throws IgniteCheckedException { - if (emptyPages != null) { - if (emptyPages.getClass() != ArrayList.class) - reuseList.put(BPlusTree.this, (FullPageId)emptyPages); - else { - List list = (List)emptyPages; - - for (int i = 0; i < list.size(); i++) - reuseList.put(BPlusTree.this, list.get(i)); - } - } + private void reuseFreePages() throws IgniteCheckedException { + // If we have a bag, then it will be processed at the upper level. + if (reuseList != null && bag == null) + reuseList.add(BPlusTree.this, this); } /** @@ -2403,13 +2453,14 @@ public abstract class BPlusTree { } /** + * @param bag Reuse bag. * @return Allocated page. */ - private Page allocatePage() throws IgniteCheckedException { - FullPageId pageId = null; + private Page allocatePage(ReuseBag bag) throws IgniteCheckedException { + FullPageId pageId = bag != null ? bag.pollFreePage() : null; - if (reuseList != null) - pageId = reuseList.take(this); + if (pageId == null && reuseList != null) + pageId = reuseList.take(this, bag); if (pageId == null) pageId = pageMem.allocatePage(cacheId, -1, PageIdAllocator.FLAG_IDX); http://git-wip-us.apache.org/repos/asf/ignite/blob/83d7a5ca/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseBag.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseBag.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseBag.java new file mode 100644 index 0000000..7fce7b7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseBag.java @@ -0,0 +1,35 @@ +/* + * 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.database.tree.reuse; + +import org.apache.ignite.internal.pagemem.FullPageId; + +/** + * Reuse bag for free index pages. + */ +public interface ReuseBag { + /** + * @param pageId Free page ID for reuse. + */ + public void addFreePage(FullPageId pageId); + + /** + * @return Free page ID for reuse or {@code null} if empty. + */ + public FullPageId pollFreePage(); +} http://git-wip-us.apache.org/repos/asf/ignite/blob/83d7a5ca/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseList.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseList.java index f494dad..eb89da9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseList.java @@ -98,21 +98,42 @@ public final class ReuseList { /** * @param client Client tree. + * @param bag Reuse bag. * @return Page ID. * @throws IgniteCheckedException If failed. */ - public FullPageId take(BPlusTree client) throws IgniteCheckedException { + public FullPageId take(BPlusTree client, ReuseBag bag) throws IgniteCheckedException { + if (trees == null) + return null; + + ReuseTree tree = tree(client); + // Remove and return page at min possible position. - return trees == null ? null : tree(client).removeCeil(MIN); + FullPageId res = tree.removeCeil(MIN, bag); + + assert res != null || tree.size() == 0; // TODO remove + + return res; } /** - * @param fullPageId Page ID. + * @param client Client tree. + * @param bag Reuse bag. * @throws IgniteCheckedException If failed. */ - public void put(BPlusTree client, FullPageId fullPageId) throws IgniteCheckedException { - assert fullPageId != null; + public void add(BPlusTree client, ReuseBag bag) throws IgniteCheckedException { + if (bag == null) + return; - tree(client).put(fullPageId); + ReuseTree tree = tree(client); + + for (;;) { + FullPageId pageId = bag.pollFreePage(); + + if (pageId == null) + break; + + tree.put(pageId, bag); + } } }