Skip to content

Commit

Permalink
gg-11427 : WIP.
Browse files Browse the repository at this point in the history
  • Loading branch information
ilantukh committed Sep 12, 2016
1 parent cd21567 commit a2e52a6
Show file tree
Hide file tree
Showing 14 changed files with 76 additions and 23 deletions.
Expand Up @@ -17,6 +17,7 @@


package org.apache.ignite.internal.processors.cache; package org.apache.ignite.internal.processors.cache;


import java.util.concurrent.atomic.AtomicLong;
import javax.cache.Cache; import javax.cache.Cache;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
Expand Down Expand Up @@ -210,6 +211,11 @@ public long entriesCount(boolean primary, boolean backup, AffinityTopologyVersio
*/ */
public long offHeapAllocatedSize(); public long offHeapAllocatedSize();


/**
* @return Global remove ID counter.
*/
public AtomicLong globalRemoveId();

// TODO GG-10884: moved from GridCacheSwapManager. // TODO GG-10884: moved from GridCacheSwapManager.
void writeAll(Iterable<GridCacheBatchSwapEntry> swapped) throws IgniteCheckedException; void writeAll(Iterable<GridCacheBatchSwapEntry> swapped) throws IgniteCheckedException;


Expand Down
Expand Up @@ -23,6 +23,7 @@
import java.util.Set; import java.util.Set;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicLong;
import javax.cache.Cache; import javax.cache.Cache;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteException;
Expand Down Expand Up @@ -95,6 +96,9 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
/** */ /** */
private static final PendingRow START_PENDING_ROW = new PendingRow(Long.MIN_VALUE, 0); private static final PendingRow START_PENDING_ROW = new PendingRow(Long.MIN_VALUE, 0);


/** */
protected final AtomicLong globalRmvId = new AtomicLong(U.currentTimeMillis() * 1000_000);

/** {@inheritDoc} */ /** {@inheritDoc} */
@Override protected void start0() throws IgniteCheckedException { @Override protected void start0() throws IgniteCheckedException {
super.start0(); super.start0();
Expand All @@ -111,10 +115,10 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
cctx.shared().database().checkpointReadLock(); cctx.shared().database().checkpointReadLock();


try { try {
reuseList = new ReuseList(cacheId, pageMem, cctx.shared().wal(), metas.rootIds(), metas.isInitNew()); reuseList = new ReuseList(cacheId, pageMem, cctx.shared().wal(), globalRmvId, metas.rootIds(), metas.isInitNew());
freeList = new FreeList(cctx, reuseList); freeList = new FreeList(cctx, reuseList, globalRmvId);


metaStore = new MetadataStorage(pageMem, cctx.shared().wal(), metaStore = new MetadataStorage(pageMem, cctx.shared().wal(), globalRmvId,
cacheId, reuseList, metas.metastoreRoot(), metas.isInitNew()); cacheId, reuseList, metas.metastoreRoot(), metas.isInitNew());


if (cctx.ttl().eagerTtlEnabled()) { if (cctx.ttl().eagerTtlEnabled()) {
Expand All @@ -125,6 +129,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
pendingEntries = new PendingEntriesTree(cctx, pendingEntries = new PendingEntriesTree(cctx,
name, name,
cctx.shared().database().pageMemory(), cctx.shared().database().pageMemory(),
globalRmvId,
rootPage.pageId().pageId(), rootPage.pageId().pageId(),
reuseList, reuseList,
rootPage.isAllocated()); rootPage.isAllocated());
Expand Down Expand Up @@ -417,6 +422,11 @@ private Iterator<CacheDataStore> cacheData(boolean primary, boolean backup, Affi
return 0; return 0;
} }


/** {@inheritDoc} */
@Override public AtomicLong globalRemoveId() {
return globalRmvId;
}

/** /**
* Clears offheap entries. * Clears offheap entries.
* *
Expand Down Expand Up @@ -674,6 +684,7 @@ protected CacheDataStore createCacheDataStore0(int p, CacheDataStore.SizeTracker
rowStore, rowStore,
cctx, cctx,
dbMgr.pageMemory(), dbMgr.pageMemory(),
globalRmvId,
rootPage.pageId().pageId(), rootPage.pageId().pageId(),
rootPage.isAllocated()); rootPage.isAllocated());


Expand Down Expand Up @@ -1010,10 +1021,11 @@ private static class CacheDataTree extends BPlusTree<KeySearchRow, DataRow> {
CacheDataRowStore rowStore, CacheDataRowStore rowStore,
GridCacheContext cctx, GridCacheContext cctx,
PageMemory pageMem, PageMemory pageMem,
AtomicLong globalRmvId,
long metaPageId, long metaPageId,
boolean initNew boolean initNew
) throws IgniteCheckedException { ) throws IgniteCheckedException {
super(name, cctx.cacheId(), pageMem, cctx.shared().wal(), metaPageId, super(name, cctx.cacheId(), pageMem, cctx.shared().wal(), globalRmvId, metaPageId,
reuseList, DataInnerIO.VERSIONS, DataLeafIO.VERSIONS); reuseList, DataInnerIO.VERSIONS, DataLeafIO.VERSIONS);


assert rowStore != null; assert rowStore != null;
Expand Down Expand Up @@ -1354,6 +1366,7 @@ private static class PendingEntriesTree extends BPlusTree<PendingRow, PendingRow
GridCacheContext cctx, GridCacheContext cctx,
String name, String name,
PageMemory pageMem, PageMemory pageMem,
AtomicLong globalRmvId,
long metaPageId, long metaPageId,
ReuseList reuseList, ReuseList reuseList,
boolean initNew) boolean initNew)
Expand All @@ -1362,6 +1375,7 @@ private static class PendingEntriesTree extends BPlusTree<PendingRow, PendingRow
cctx.cacheId(), cctx.cacheId(),
pageMem, pageMem,
cctx.shared().wal(), cctx.shared().wal(),
globalRmvId,
metaPageId, metaPageId,
reuseList, reuseList,
PendingEntryInnerIO.VERSIONS, PendingEntryInnerIO.VERSIONS,
Expand Down
Expand Up @@ -19,6 +19,7 @@


import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteException;
import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.FullPageId;
Expand Down Expand Up @@ -65,6 +66,7 @@ public class MetadataStorage implements MetaStore {
public MetadataStorage( public MetadataStorage(
final PageMemory pageMem, final PageMemory pageMem,
final IgniteWriteAheadLogManager wal, final IgniteWriteAheadLogManager wal,
final AtomicLong globalRmvId,
final int cacheId, final int cacheId,
final ReuseList reuseList, final ReuseList reuseList,
final long rootPageId, final long rootPageId,
Expand All @@ -75,7 +77,7 @@ public MetadataStorage(
this.cacheId = cacheId; this.cacheId = cacheId;
this.reuseList = reuseList; this.reuseList = reuseList;


metaTree = new MetaTree(cacheId, pageMem, wal, rootPageId, metaTree = new MetaTree(cacheId, pageMem, wal, globalRmvId, rootPageId,
reuseList, MetaStoreInnerIO.VERSIONS, MetaStoreLeafIO.VERSIONS, initNew); reuseList, MetaStoreInnerIO.VERSIONS, MetaStoreLeafIO.VERSIONS, initNew);
} }
catch (IgniteCheckedException e) { catch (IgniteCheckedException e) {
Expand Down Expand Up @@ -152,13 +154,14 @@ private MetaTree(
final int cacheId, final int cacheId,
final PageMemory pageMem, final PageMemory pageMem,
final IgniteWriteAheadLogManager wal, final IgniteWriteAheadLogManager wal,
final AtomicLong globalRmvId,
final long metaPageId, final long metaPageId,
final ReuseList reuseList, final ReuseList reuseList,
final IOVersions<? extends BPlusInnerIO<IndexItem>> innerIos, final IOVersions<? extends BPlusInnerIO<IndexItem>> innerIos,
final IOVersions<? extends BPlusLeafIO<IndexItem>> leafIos, final IOVersions<? extends BPlusLeafIO<IndexItem>> leafIos,
final boolean initNew final boolean initNew
) throws IgniteCheckedException { ) throws IgniteCheckedException {
super(treeName("meta", "Meta"), cacheId, pageMem, wal, metaPageId, reuseList, innerIos, leafIos); super(treeName("meta", "Meta"), cacheId, pageMem, wal, globalRmvId, metaPageId, reuseList, innerIos, leafIos);


if (initNew) if (initNew)
initNew(); initNew();
Expand Down
Expand Up @@ -18,6 +18,7 @@
package org.apache.ignite.internal.processors.cache.database.freelist; package org.apache.ignite.internal.processors.cache.database.freelist;


import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.pagemem.Page; import org.apache.ignite.internal.pagemem.Page;
import org.apache.ignite.internal.pagemem.PageIdAllocator; import org.apache.ignite.internal.pagemem.PageIdAllocator;
Expand Down Expand Up @@ -65,6 +66,9 @@ public class FreeList {
/** */ /** */
private final ConcurrentHashMap8<Integer, GridFutureAdapter<FreeTree>> trees = new ConcurrentHashMap8<>(); private final ConcurrentHashMap8<Integer, GridFutureAdapter<FreeTree>> trees = new ConcurrentHashMap8<>();


/** */
private final AtomicLong globalRmvId;

/** */ /** */
private final PageHandler<CacheDataRow, Integer> writeRow = new PageHandler<CacheDataRow, Integer>() { private final PageHandler<CacheDataRow, Integer> writeRow = new PageHandler<CacheDataRow, Integer>() {
@Override public Integer run(long pageId, Page page, ByteBuffer buf, CacheDataRow row, int written) @Override public Integer run(long pageId, Page page, ByteBuffer buf, CacheDataRow row, int written)
Expand Down Expand Up @@ -207,7 +211,7 @@ private int addRowFragment(
* @param reuseList Reuse list. * @param reuseList Reuse list.
* @param cctx Cache context. * @param cctx Cache context.
*/ */
public FreeList(GridCacheContext<?, ?> cctx, ReuseList reuseList) { public FreeList(GridCacheContext<?, ?> cctx, ReuseList reuseList, AtomicLong globalRmvId) {
assert cctx != null; assert cctx != null;


this.cctx = cctx; this.cctx = cctx;
Expand All @@ -219,6 +223,7 @@ public FreeList(GridCacheContext<?, ?> cctx, ReuseList reuseList) {
assert pageMem != null; assert pageMem != null;


this.reuseList = reuseList; this.reuseList = reuseList;
this.globalRmvId = globalRmvId;
} }


/** /**
Expand Down Expand Up @@ -256,7 +261,7 @@ private FreeTree tree(Integer partId) throws IgniteCheckedException {


final RootPage rootPage = cctx.offheap().meta().getOrAllocateForTree(idxName); final RootPage rootPage = cctx.offheap().meta().getOrAllocateForTree(idxName);


fut.onDone(new FreeTree(idxName, reuseList, cctx.cacheId(), partId, pageMem, wal, fut.onDone(new FreeTree(idxName, reuseList, cctx.cacheId(), partId, pageMem, wal, globalRmvId,
rootPage.pageId().pageId(), rootPage.isAllocated())); rootPage.pageId().pageId(), rootPage.isAllocated()));
} }
} }
Expand Down
Expand Up @@ -18,6 +18,7 @@
package org.apache.ignite.internal.processors.cache.database.freelist; package org.apache.ignite.internal.processors.cache.database.freelist;


import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.PageIdUtils;
import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.PageMemory;
Expand Down Expand Up @@ -53,10 +54,11 @@ public FreeTree(
int partId, int partId,
PageMemory pageMem, PageMemory pageMem,
IgniteWriteAheadLogManager wal, IgniteWriteAheadLogManager wal,
AtomicLong globalRmvId,
long metaPageId, long metaPageId,
boolean initNew boolean initNew
) throws IgniteCheckedException { ) throws IgniteCheckedException {
super(name, cacheId, pageMem, wal, metaPageId, reuseList, FreeInnerIO.VERSIONS, FreeLeafIO.VERSIONS); super(name, cacheId, pageMem, wal, globalRmvId, metaPageId, reuseList, FreeInnerIO.VERSIONS, FreeLeafIO.VERSIONS);


this.partId = partId; this.partId = partId;


Expand Down
Expand Up @@ -123,7 +123,8 @@ public abstract class BPlusTree<L, T extends L> {
private final IOVersions<? extends BPlusLeafIO<L>> leafIos; private final IOVersions<? extends BPlusLeafIO<L>> leafIos;


/** */ /** */
private final AtomicLong globalRmvId = new AtomicLong(U.currentTimeMillis() * 1000_000); // TODO init from WAL? // private final AtomicLong globalRmvId = new AtomicLong(U.currentTimeMillis() * 1000_000); // TODO init from WAL?
private final AtomicLong globalRmvId;


/** */ /** */
private final GridTreePrinter<Long> treePrinter = new GridTreePrinter<Long>() { private final GridTreePrinter<Long> treePrinter = new GridTreePrinter<Long>() {
Expand Down Expand Up @@ -606,6 +607,7 @@ public BPlusTree(
int cacheId, int cacheId,
PageMemory pageMem, PageMemory pageMem,
IgniteWriteAheadLogManager wal, IgniteWriteAheadLogManager wal,
AtomicLong globalRmvId,
long metaPageId, long metaPageId,
ReuseList reuseList, ReuseList reuseList,
IOVersions<? extends BPlusInnerIO<L>> innerIos, IOVersions<? extends BPlusInnerIO<L>> innerIos,
Expand All @@ -630,6 +632,7 @@ public BPlusTree(
this.metaPageId = metaPageId; this.metaPageId = metaPageId;
this.reuseList = reuseList; this.reuseList = reuseList;
this.wal = wal; this.wal = wal;
this.globalRmvId = globalRmvId;
} }


/** /**
Expand Down
Expand Up @@ -25,7 +25,10 @@ public class PagePartMetaIO extends PageIO {
private static final int SIZE_OFF = PageIO.COMMON_HEADER_END; private static final int SIZE_OFF = PageIO.COMMON_HEADER_END;


/** */ /** */
private static final int UPDATE_CNTR_OFF = SIZE_OFF + 4; private static final int UPDATE_CNTR_OFF = SIZE_OFF + 8;

/** */
private static final int GLOBAL_RMV_ID_OFF = UPDATE_CNTR_OFF + 8;


/** */ /** */
public static final IOVersions<PagePartMetaIO> VERSIONS = new IOVersions<>( public static final IOVersions<PagePartMetaIO> VERSIONS = new IOVersions<>(
Expand All @@ -36,12 +39,12 @@ public PagePartMetaIO(int ver) {
super(T_PART_META, ver); super(T_PART_META, ver);
} }


public int getSize(ByteBuffer buf) { public long getSize(ByteBuffer buf) {
return buf.getInt(SIZE_OFF); return buf.getLong(SIZE_OFF);
} }


public void setSize(ByteBuffer buf, int size) { public void setSize(ByteBuffer buf, long size) {
buf.putInt(SIZE_OFF, size); buf.putLong(SIZE_OFF, size);
} }


public long getUpdateCounter(ByteBuffer buf) { public long getUpdateCounter(ByteBuffer buf) {
Expand All @@ -51,4 +54,12 @@ public long getUpdateCounter(ByteBuffer buf) {
public void setUpdateCounter(ByteBuffer buf, long cntr) { public void setUpdateCounter(ByteBuffer buf, long cntr) {
buf.putLong(UPDATE_CNTR_OFF, cntr); buf.putLong(UPDATE_CNTR_OFF, cntr);
} }

public long getGlobalRemoveId(ByteBuffer buf) {
return buf.getLong(GLOBAL_RMV_ID_OFF);
}

public void setGlobalRemoveId(ByteBuffer buf, long rmvId) {
buf.putLong(GLOBAL_RMV_ID_OFF, rmvId);
}
} }
Expand Up @@ -17,6 +17,7 @@


package org.apache.ignite.internal.processors.cache.database.tree.reuse; package org.apache.ignite.internal.processors.cache.database.tree.reuse;


import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.PageMemory;
import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
Expand All @@ -40,7 +41,7 @@ public final class ReuseList {
* @param initNew Init new flag. * @param initNew Init new flag.
* @throws IgniteCheckedException If failed. * @throws IgniteCheckedException If failed.
*/ */
public ReuseList(int cacheId, PageMemory pageMem, IgniteWriteAheadLogManager wal, long[] rooIds, public ReuseList(int cacheId, PageMemory pageMem, IgniteWriteAheadLogManager wal, AtomicLong globalRmvId, long[] rooIds,
boolean initNew) throws IgniteCheckedException { boolean initNew) throws IgniteCheckedException {
A.ensure(rooIds.length > 1, "Segments must be greater than 1."); A.ensure(rooIds.length > 1, "Segments must be greater than 1.");


Expand All @@ -49,7 +50,7 @@ public ReuseList(int cacheId, PageMemory pageMem, IgniteWriteAheadLogManager wal
for (int i = 0; i < rooIds.length; i++) { for (int i = 0; i < rooIds.length; i++) {
String idxName = BPlusTree.treeName("s" + i, "Reuse"); String idxName = BPlusTree.treeName("s" + i, "Reuse");


trees[i] = new ReuseTree(idxName, this, cacheId, pageMem, wal, rooIds[i], initNew); trees[i] = new ReuseTree(idxName, this, cacheId, pageMem, wal, globalRmvId, rooIds[i], initNew);
} }
} }


Expand Down
Expand Up @@ -18,6 +18,7 @@
package org.apache.ignite.internal.processors.cache.database.tree.reuse; package org.apache.ignite.internal.processors.cache.database.tree.reuse;


import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.PageIdUtils;
import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.PageMemory;
Expand Down Expand Up @@ -47,10 +48,11 @@ public ReuseTree(
int cacheId, int cacheId,
PageMemory pageMem, PageMemory pageMem,
IgniteWriteAheadLogManager wal, IgniteWriteAheadLogManager wal,
AtomicLong globalRmvId,
long metaPageId, long metaPageId,
boolean initNew boolean initNew
) throws IgniteCheckedException { ) throws IgniteCheckedException {
super(name, cacheId, pageMem, wal, metaPageId, reuseList, ReuseInnerIO.VERSIONS, ReuseLeafIO.VERSIONS); super(name, cacheId, pageMem, wal, globalRmvId, metaPageId, reuseList, ReuseInnerIO.VERSIONS, ReuseLeafIO.VERSIONS);


if (initNew) if (initNew)
initNew(); initNew();
Expand Down
Expand Up @@ -18,6 +18,7 @@
package org.apache.ignite.internal.processors.query.h2.database; package org.apache.ignite.internal.processors.query.h2.database;


import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.FullPageId;
import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.PageMemory;
Expand Down Expand Up @@ -53,11 +54,12 @@ public H2Tree(
int cacheId, int cacheId,
PageMemory pageMem, PageMemory pageMem,
IgniteWriteAheadLogManager wal, IgniteWriteAheadLogManager wal,
AtomicLong globalRmvId,
H2RowFactory rowStore, H2RowFactory rowStore,
long metaPageId, long metaPageId,
boolean initNew boolean initNew
) throws IgniteCheckedException { ) throws IgniteCheckedException {
super(name, cacheId, pageMem, wal, metaPageId, reuseList, H2InnerIO.VERSIONS, H2LeafIO.VERSIONS); super(name, cacheId, pageMem, wal, globalRmvId, metaPageId, reuseList, H2InnerIO.VERSIONS, H2LeafIO.VERSIONS);


assert rowStore != null; assert rowStore != null;


Expand Down
Expand Up @@ -82,7 +82,8 @@ public H2TreeIndex(
RootPage page = cctx.offheap().meta().getOrAllocateForTree(name); RootPage page = cctx.offheap().meta().getOrAllocateForTree(name);


tree = new H2Tree(name, cctx.offheap().reuseList(), cctx.cacheId(), tree = new H2Tree(name, cctx.offheap().reuseList(), cctx.cacheId(),
dbMgr.pageMemory(), cctx.shared().wal(), tbl.rowFactory(), page.pageId().pageId(), page.isAllocated()) { dbMgr.pageMemory(), cctx.shared().wal(), cctx.offheap().globalRemoveId(),
tbl.rowFactory(), page.pageId().pageId(), page.isAllocated()) {
@Override protected int compare(BPlusIO<SearchRow> io, ByteBuffer buf, int idx, SearchRow row) @Override protected int compare(BPlusIO<SearchRow> io, ByteBuffer buf, int idx, SearchRow row)
throws IgniteCheckedException { throws IgniteCheckedException {
return compareRows(getRow(io, buf, idx), row); return compareRows(getRow(io, buf, idx), row);
Expand Down
Expand Up @@ -19,6 +19,7 @@


import java.util.Map; import java.util.Map;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.PageMemory;
Expand All @@ -35,7 +36,7 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
/** {@inheritDoc} */ /** {@inheritDoc} */
@Override protected ReuseList createReuseList(int cacheId, PageMemory pageMem, long[] rootIds, boolean initNew) @Override protected ReuseList createReuseList(int cacheId, PageMemory pageMem, long[] rootIds, boolean initNew)
throws IgniteCheckedException { throws IgniteCheckedException {
return new ReuseList(cacheId, pageMem, null, rootIds, initNew); return new ReuseList(cacheId, pageMem, null, new AtomicLong(), rootIds, initNew);
} }


/** /**
Expand Down
Expand Up @@ -26,6 +26,7 @@
import java.util.TreeMap; import java.util.TreeMap;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider; import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.FullPageId;
Expand Down Expand Up @@ -764,7 +765,7 @@ protected static class TestTree extends BPlusTree<Long, Long> {
*/ */
public TestTree(ReuseList reuseList, boolean canGetRow, int cacheId, PageMemory pageMem, long metaPageId) public TestTree(ReuseList reuseList, boolean canGetRow, int cacheId, PageMemory pageMem, long metaPageId)
throws IgniteCheckedException { throws IgniteCheckedException {
super("test", cacheId, pageMem, null, metaPageId, reuseList, super("test", cacheId, pageMem, null, new AtomicLong(), metaPageId, reuseList,
new IOVersions<>(new LongInnerIO(canGetRow)), new IOVersions<>(new LongLeafIO())); new IOVersions<>(new LongInnerIO(canGetRow)), new IOVersions<>(new LongLeafIO()));


initNew(); initNew();
Expand Down

0 comments on commit a2e52a6

Please sign in to comment.